From f382a91d54d27b24a51cb08a7a9a2ed130fff831 Mon Sep 17 00:00:00 2001 From: fastio Date: Thu, 21 Jul 2022 10:49:09 +0800 Subject: [PATCH 001/188] Auto optimize partition in background if enabled --- src/Core/Settings.h | 1 + src/Storages/MergeTree/MergeTreeSettings.h | 4 +- .../ReplicatedMergeTreeRestartingThread.cpp | 2 + src/Storages/StorageMergeTree.cpp | 63 ++++++++++++++++ src/Storages/StorageMergeTree.h | 11 +++ src/Storages/StorageReplicatedMergeTree.cpp | 55 ++++++++++++-- src/Storages/StorageReplicatedMergeTree.h | 17 ++++- .../test_auto_optimize_partitions/__init__.py | 0 .../configs/zookeeper_config.xml | 8 ++ .../test_auto_optimize_partitions/test.py | 75 +++++++++++++++++++ 10 files changed, 229 insertions(+), 7 deletions(-) create mode 100644 tests/integration/test_auto_optimize_partitions/__init__.py create mode 100644 tests/integration/test_auto_optimize_partitions/configs/zookeeper_config.xml create mode 100644 tests/integration/test_auto_optimize_partitions/test.py diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bda72f089eb..99ad00df70e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -634,6 +634,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) MAKE_OBSOLETE(M, Bool, allow_experimental_map_type, true) \ MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60) \ MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \ + MAKE_OBSOLETE(M, UInt64, auto_optimize_partition_interval_seconds, 600) \ MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \ MAKE_OBSOLETE(M, Bool, allow_experimental_projection_optimization, true) \ diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index d8dfa017b7f..6b780a4d718 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -62,6 +62,7 @@ struct Settings; M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \ M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ M(UInt64, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds, 1ULL * 3600 * 24 * 30, "Remove old broken detached parts in the background if they remained intouched for a specified by this setting period of time.", 0) \ + M(UInt64, auto_optimize_partition_interval_seconds, 600, "The period of executing the auto optimize partitions in background.", 0) \ M(UInt64, merge_tree_enable_clear_old_broken_detached, false, "Enable clearing old broken detached parts operation in background.", 0) \ M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \ \ @@ -152,7 +153,8 @@ struct Settings; M(UInt64, replicated_max_parallel_sends_for_table, 0, "Obsolete setting, does nothing.", 0) \ M(UInt64, replicated_max_parallel_fetches, 0, "Obsolete setting, does nothing.", 0) \ M(UInt64, replicated_max_parallel_fetches_for_table, 0, "Obsolete setting, does nothing.", 0) \ - M(Bool, write_final_mark, true, "Obsolete setting, does nothing.", 0) + M(Bool, write_final_mark, true, "Obsolete setting, does nothing.", 0) \ + M(UInt64, auto_optimize_partition_after_seconds, 0, "The number seconds of which to do automatically optimization for partitions. Zero means disable.", 0) /// Settings that should not change after the creation of a table. /// NOLINTNEXTLINE #define APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(M) \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index dfb5eb0bd69..053e2031183 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -157,6 +157,8 @@ bool ReplicatedMergeTreeRestartingThread::runImpl() storage.mutations_updating_task->activateAndSchedule(); storage.mutations_finalizing_task->activateAndSchedule(); storage.merge_selecting_task->activateAndSchedule(); + if (storage.auto_optimize_partition_task) + storage.auto_optimize_partition_task->activateAndSchedule(); storage.cleanup_thread.start(); return true; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 6825698f006..7d33c4b16f3 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -765,6 +765,63 @@ void StorageMergeTree::loadMutations() increment.value = std::max(increment.value.load(), current_mutations_by_version.rbegin()->first); } +std::shared_ptr StorageMergeTree::selectOnePartitionToOptimize( + const StorageMetadataPtr & metadata_snapshot, + TableLockHolder & table_lock_holder, + std::unique_lock & lock, + const MergeTreeTransactionPtr & txn, + bool optimize_skip_merged_partitions) +{ + // Select the `best partition to merge. + std::unordered_map partition_parts_sum_diff; + ssize_t base = time(nullptr) - getSettings()->auto_optimize_partition_after_seconds; + auto data_parts = getDataPartsForInternalUsage(); + for (auto part : data_parts) + { + if (part->modification_time < base) + partition_parts_sum_diff[part->info.partition_id] += (base - part->modification_time); + } + String best_partition_id; + Int32 max_diff = 0; + for (auto [partition_id, diff] : partition_parts_sum_diff) + { + if (diff > max_diff) + { + best_partition_id = partition_id; + max_diff = diff; + } + } + if (best_partition_id.empty()) + { + return nullptr; + } + // Merge the selected partition. + String disable_reason; + SelectPartsDecision select_decision; + auto merge_entry = selectPartsToMerge( + metadata_snapshot, + true, + best_partition_id, + true, + &disable_reason, + table_lock_holder, + lock, + txn, + optimize_skip_merged_partitions, + &select_decision); + + if (select_decision == SelectPartsDecision::NOTHING_TO_MERGE) + return nullptr; + if (!merge_entry) + { + constexpr const char * message = "Cannot OPTIMIZE table: {} in background"; + if (disable_reason.empty()) + disable_reason = "unknown reason"; + LOG_INFO(log, fmt::runtime(message), disable_reason); + } + return merge_entry; +} + std::shared_ptr StorageMergeTree::selectPartsToMerge( const StorageMetadataPtr & metadata_snapshot, bool aggressive, @@ -1150,6 +1207,12 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign if (!merge_entry) mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr, share_lock, lock, were_some_mutations_skipped); + if (getSettings()->auto_optimize_partition_after_seconds + && !merge_entry && !mutate_entry + && time_after_previous_optimize_one_partition.compareAndRestartDeferred(getSettings()->auto_optimize_partition_interval_seconds)) + { + merge_entry = selectOnePartitionToOptimize(metadata_snapshot, share_lock, lock, txn); + } has_mutations = !current_mutations_by_version.empty(); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 8ca8ab5d11e..d5e4f978112 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -133,6 +133,8 @@ private: AtomicStopwatch time_after_previous_cleanup_temporary_directories; /// For clearOldBrokenDetachedParts AtomicStopwatch time_after_previous_cleanup_broken_detached_parts; + /// For optimizeOnePartition; + AtomicStopwatch time_after_previous_optimize_one_partition; /// Mutex for parts currently processing in background /// merging (also with TTL), mutating or moving. @@ -172,6 +174,15 @@ private: String * out_disable_reason = nullptr, bool optimize_skip_merged_partitions = false); + /** Determines what parts which within one partition should be merged. + */ + std::shared_ptr selectOnePartitionToOptimize( + const StorageMetadataPtr & metadata_snapshot, + TableLockHolder & table_lock_holder, + std::unique_lock & lock, + const MergeTreeTransactionPtr & txn, + bool optimize_skip_merged_partitions = false); + /// Make part state outdated and queue it to remove without timeout /// If force, then stop merges and block them until part state became outdated. Throw exception if part doesn't exists /// If not force, then take merges selector and check that part is not participating in background operations. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 520b5534fe3..e7e3a1ad1b2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -288,9 +288,14 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( merge_selecting_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mergeSelectingTask)", [this] { mergeSelectingTask(); }); + if (getSettings()->auto_optimize_partition_after_seconds) + auto_optimize_partition_task = getContext()->getSchedulePool().createTask( + getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::autoOptimizePartitionTask)", [this] { autoOptimizePartitionTask(); }); /// Will be activated if we win leader election. merge_selecting_task->deactivate(); + if (auto_optimize_partition_task) + auto_optimize_partition_task->deactivate(); mutations_finalizing_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mutationsFinalizingTask)", [this] { mutationsFinalizingTask(); }); @@ -4447,10 +4452,30 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con local_context); } +void StorageReplicatedMergeTree::autoOptimizePartitionTask() +{ + if (!is_leader || !getSettings()->auto_optimize_partition_after_seconds) + return; + auto table_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + try + { + optimizeImpl(nullptr, nullptr, nullptr, true, true, Names{}, table_lock, getContext(), true); + } + catch (const Exception & e) + { + LOG_ERROR(log, "Can't optimize partitions automatically for table: {}, reason: {}", getStorageID().getNameForLogs(), e.displayText()); + } + catch (...) + { + LOG_ERROR(log, "There is a problem when optimizing table: {}, reason: {}", getStorageID().getNameForLogs(), getCurrentExceptionMessage(true)); + } + if (auto_optimize_partition_task) + auto_optimize_partition_task->scheduleAfter(getSettings()->auto_optimize_partition_interval_seconds); +} bool StorageReplicatedMergeTree::optimize( - const ASTPtr &, - const StorageMetadataPtr &, + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, const ASTPtr & partition, bool final, bool deduplicate, @@ -4460,7 +4485,20 @@ bool StorageReplicatedMergeTree::optimize( /// NOTE: exclusive lock cannot be used here, since this may lead to deadlock (see comments below), /// but it should be safe to use non-exclusive to avoid dropping parts that may be required for processing queue. auto table_lock = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef().lock_acquire_timeout); + return optimizeImpl(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, table_lock, query_context, false); +} +bool StorageReplicatedMergeTree::optimizeImpl( + const ASTPtr &, + const StorageMetadataPtr &, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Names & deduplicate_by_columns, + TableLockHolder & table_lock, + ContextPtr query_context, + bool auto_optimize_in_background) +{ assertNotReadonly(); if (!is_leader) @@ -4558,8 +4596,12 @@ bool StorageReplicatedMergeTree::optimize( DataPartsVector data_parts = getVisibleDataPartsVector(query_context); std::unordered_set partition_ids; + ssize_t baseline = time(nullptr) - storage_settings_ptr->auto_optimize_partition_after_seconds; for (const DataPartPtr & part : data_parts) - partition_ids.emplace(part->info.partition_id); + { + if (!auto_optimize_in_background || part->modification_time < baseline) + partition_ids.emplace(part->info.partition_id); + } for (const String & partition_id : partition_ids) { @@ -4578,8 +4620,11 @@ bool StorageReplicatedMergeTree::optimize( table_lock.reset(); - for (auto & merge_entry : merge_entries) - waitForLogEntryToBeProcessedIfNecessary(merge_entry, query_context); + if (!auto_optimize_in_background) + { + for (auto & merge_entry : merge_entries) + waitForLogEntryToBeProcessedIfNecessary(merge_entry, query_context); + } return assigned; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 18b9ef54777..d1cd9a70b96 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -325,7 +325,16 @@ public: private: std::atomic_bool are_restoring_replica {false}; - + bool optimizeImpl( + const ASTPtr &, + const StorageMetadataPtr &, + const ASTPtr & partition, + bool final, + bool deduplicate, + const Names & deduplicate_by_columns, + TableLockHolder & table_lock, + ContextPtr query_context, + bool auto_optimize_in_background); /// Get a sequential consistent view of current parts. ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const; @@ -433,6 +442,9 @@ private: /// A task that marks finished mutations as done. BackgroundSchedulePool::TaskHolder mutations_finalizing_task; + /// A task that optimize partitions automatically background if enabled. + BackgroundSchedulePool::TaskHolder auto_optimize_partition_task { nullptr }; + /// A thread that removes old parts, log entries, and blocks. ReplicatedMergeTreeCleanupThread cleanup_thread; @@ -566,6 +578,9 @@ private: void mutationsUpdatingTask(); + /** Auto optimize partitions + */ + void autoOptimizePartitionTask(); /** Clone data from another replica. * If replica can not be cloned throw Exception. */ diff --git a/tests/integration/test_auto_optimize_partitions/__init__.py b/tests/integration/test_auto_optimize_partitions/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_auto_optimize_partitions/configs/zookeeper_config.xml b/tests/integration/test_auto_optimize_partitions/configs/zookeeper_config.xml new file mode 100644 index 00000000000..18412349228 --- /dev/null +++ b/tests/integration/test_auto_optimize_partitions/configs/zookeeper_config.xml @@ -0,0 +1,8 @@ + + + + zoo1 + 2181 + + + diff --git a/tests/integration/test_auto_optimize_partitions/test.py b/tests/integration/test_auto_optimize_partitions/test.py new file mode 100644 index 00000000000..c1c848b4cad --- /dev/null +++ b/tests/integration/test_auto_optimize_partitions/test.py @@ -0,0 +1,75 @@ +import pytest +import time; +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", main_configs=["configs/zookeeper_config.xml"], with_zookeeper=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + +def test_without_auto_optimize_merge_tree(start_cluster): + node.query( + "CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i;" + ) + node.query("INSERT INTO test SELECT 1") + node.query("INSERT INTO test SELECT 2") + node.query("INSERT INTO test SELECT 3") + + + expected = TSV('''all_0_0_0\t1\tall_1_1_0\t2\all_2_2_0\t3''') + assert TSV(node.query('SELECT _part, * FROM test')) == expected + + time.sleep(30) + + expected = TSV('''all_0_0_0\t1\tall_1_1_0\t2\all_2_2_0\t3''') + assert TSV(node.query('SELECT _part, * FROM test')) == expected + + node.query("DROP TABLE test;") + +def test_auto_optimize_merge_tree(start_cluster): + node.query( + "CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i SETTINGS auto_optimize_partition_after_seconds=30;" + ) + node.query("INSERT INTO test SELECT 1") + node.query("INSERT INTO test SELECT 2") + node.query("INSERT INTO test SELECT 3") + + + expected = TSV('''all_0_0_0\t1\tall_1_1_0\t2\all_2_2_0\t3''') + assert TSV(node.query('SELECT _part, * FROM test')) == expected + + time.sleep(30) + + expected = TSV('''all_0_2_1\t1\tall_0_2_1\t2\all_0_2_1\t3''') + assert TSV(node.query('SELECT _part, * FROM test')) == expected + + node.query("DROP TABLE test;") + +def test_auto_optimize_replicated_merge_tree(start_cluster): + node.query( + "CREATE TABLE test (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') ORDER BY i SETTINGS auto_optimize_partition_after_seconds=30;" + ) + node.query("INSERT INTO test SELECT 1") + node.query("INSERT INTO test SELECT 2") + node.query("INSERT INTO test SELECT 3") + + + expected = TSV('''all_0_0_0\t1\tall_1_1_0\t2\all_2_2_0\t3''') + assert TSV(node.query('SELECT _part, * FROM test')) == expected + + time.sleep(30) + + expected = TSV('''all_0_2_1\t1\tall_0_2_1\t2\all_0_2_1\t3''') + assert TSV(node.query('SELECT _part, * FROM test')) == expected + + node.query("DROP TABLE test;") From 8210e8c58c71bfe4c7ecfcc03fdc8861078022dd Mon Sep 17 00:00:00 2001 From: fastio Date: Mon, 25 Jul 2022 17:46:55 +0800 Subject: [PATCH 002/188] fix test cases --- .../configs/merge_tree.xml | 5 +++ .../test_auto_optimize_partitions/test.py | 33 +++++++------------ 2 files changed, 17 insertions(+), 21 deletions(-) create mode 100644 tests/integration/test_auto_optimize_partitions/configs/merge_tree.xml diff --git a/tests/integration/test_auto_optimize_partitions/configs/merge_tree.xml b/tests/integration/test_auto_optimize_partitions/configs/merge_tree.xml new file mode 100644 index 00000000000..9c3da3c99ac --- /dev/null +++ b/tests/integration/test_auto_optimize_partitions/configs/merge_tree.xml @@ -0,0 +1,5 @@ + + + 2 + + diff --git a/tests/integration/test_auto_optimize_partitions/test.py b/tests/integration/test_auto_optimize_partitions/test.py index c1c848b4cad..d07c12a9416 100644 --- a/tests/integration/test_auto_optimize_partitions/test.py +++ b/tests/integration/test_auto_optimize_partitions/test.py @@ -5,7 +5,7 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node", main_configs=["configs/zookeeper_config.xml"], with_zookeeper=True) +node = cluster.add_instance("node", main_configs=["configs/zookeeper_config.xml", "configs/merge_tree.xml"], with_zookeeper=True) @pytest.fixture(scope="module") @@ -26,50 +26,41 @@ def test_without_auto_optimize_merge_tree(start_cluster): node.query("INSERT INTO test SELECT 3") - expected = TSV('''all_0_0_0\t1\tall_1_1_0\t2\all_2_2_0\t3''') - assert TSV(node.query('SELECT _part, * FROM test')) == expected + time.sleep(5) - time.sleep(30) - - expected = TSV('''all_0_0_0\t1\tall_1_1_0\t2\all_2_2_0\t3''') - assert TSV(node.query('SELECT _part, * FROM test')) == expected + expected = TSV('''3\n''') + assert TSV(node.query("SELECT count(*) FROM system.parts where table='test' and active=1")) == expected node.query("DROP TABLE test;") def test_auto_optimize_merge_tree(start_cluster): node.query( - "CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i SETTINGS auto_optimize_partition_after_seconds=30;" + "CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i SETTINGS auto_optimize_partition_after_seconds=5;" ) node.query("INSERT INTO test SELECT 1") node.query("INSERT INTO test SELECT 2") node.query("INSERT INTO test SELECT 3") - expected = TSV('''all_0_0_0\t1\tall_1_1_0\t2\all_2_2_0\t3''') - assert TSV(node.query('SELECT _part, * FROM test')) == expected + time.sleep(10) - time.sleep(30) - - expected = TSV('''all_0_2_1\t1\tall_0_2_1\t2\all_0_2_1\t3''') - assert TSV(node.query('SELECT _part, * FROM test')) == expected + expected = TSV('''1\n''') + assert TSV(node.query("SELECT count(*) FROM system.parts where table='test' and active=1")) == expected node.query("DROP TABLE test;") def test_auto_optimize_replicated_merge_tree(start_cluster): node.query( - "CREATE TABLE test (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') ORDER BY i SETTINGS auto_optimize_partition_after_seconds=30;" + "CREATE TABLE test (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') ORDER BY i SETTINGS auto_optimize_partition_after_seconds=5;" ) node.query("INSERT INTO test SELECT 1") node.query("INSERT INTO test SELECT 2") node.query("INSERT INTO test SELECT 3") - expected = TSV('''all_0_0_0\t1\tall_1_1_0\t2\all_2_2_0\t3''') - assert TSV(node.query('SELECT _part, * FROM test')) == expected + time.sleep(10) - time.sleep(30) - - expected = TSV('''all_0_2_1\t1\tall_0_2_1\t2\all_0_2_1\t3''') - assert TSV(node.query('SELECT _part, * FROM test')) == expected + expected = TSV('''1\n''') + assert TSV(node.query("SELECT count(*) FROM system.parts where table='test' and active=1")) == expected node.query("DROP TABLE test;") From 7b4c90784ea3d79d03b87996940b2420e931a624 Mon Sep 17 00:00:00 2001 From: fastio Date: Mon, 25 Jul 2022 20:06:52 +0800 Subject: [PATCH 003/188] format test case codes --- .../test_auto_optimize_partitions/test.py | 32 ++++++++++++------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_auto_optimize_partitions/test.py b/tests/integration/test_auto_optimize_partitions/test.py index d07c12a9416..f829a062a5f 100644 --- a/tests/integration/test_auto_optimize_partitions/test.py +++ b/tests/integration/test_auto_optimize_partitions/test.py @@ -1,11 +1,14 @@ import pytest -import time; +import time from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node", main_configs=["configs/zookeeper_config.xml", "configs/merge_tree.xml"], with_zookeeper=True) +node = cluster.add_instance( + "node", + main_configs=["configs/zookeeper_config.xml", "configs/merge_tree.xml"], + with_zookeeper=True) @pytest.fixture(scope="module") @@ -17,22 +20,24 @@ def start_cluster(): finally: cluster.shutdown() + def test_without_auto_optimize_merge_tree(start_cluster): - node.query( - "CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i;" - ) + node.query("CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i;") node.query("INSERT INTO test SELECT 1") node.query("INSERT INTO test SELECT 2") node.query("INSERT INTO test SELECT 3") - time.sleep(5) expected = TSV('''3\n''') - assert TSV(node.query("SELECT count(*) FROM system.parts where table='test' and active=1")) == expected + assert TSV( + node.query( + "SELECT count(*) FROM system.parts where table='test' and active=1" + )) == expected node.query("DROP TABLE test;") + def test_auto_optimize_merge_tree(start_cluster): node.query( "CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i SETTINGS auto_optimize_partition_after_seconds=5;" @@ -41,14 +46,17 @@ def test_auto_optimize_merge_tree(start_cluster): node.query("INSERT INTO test SELECT 2") node.query("INSERT INTO test SELECT 3") - time.sleep(10) expected = TSV('''1\n''') - assert TSV(node.query("SELECT count(*) FROM system.parts where table='test' and active=1")) == expected + assert TSV( + node.query( + "SELECT count(*) FROM system.parts where table='test' and active=1" + )) == expected node.query("DROP TABLE test;") + def test_auto_optimize_replicated_merge_tree(start_cluster): node.query( "CREATE TABLE test (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') ORDER BY i SETTINGS auto_optimize_partition_after_seconds=5;" @@ -57,10 +65,12 @@ def test_auto_optimize_replicated_merge_tree(start_cluster): node.query("INSERT INTO test SELECT 2") node.query("INSERT INTO test SELECT 3") - time.sleep(10) expected = TSV('''1\n''') - assert TSV(node.query("SELECT count(*) FROM system.parts where table='test' and active=1")) == expected + assert TSV( + node.query( + "SELECT count(*) FROM system.parts where table='test' and active=1" + )) == expected node.query("DROP TABLE test;") From 6153bd453b71d53696354122204a7270bd50ba1a Mon Sep 17 00:00:00 2001 From: fastio Date: Tue, 26 Jul 2022 14:35:38 +0800 Subject: [PATCH 004/188] fix compile error --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7d33c4b16f3..9b41e4dae53 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -776,7 +776,7 @@ std::shared_ptr StorageMergeTree::selectOnePartitionTo std::unordered_map partition_parts_sum_diff; ssize_t base = time(nullptr) - getSettings()->auto_optimize_partition_after_seconds; auto data_parts = getDataPartsForInternalUsage(); - for (auto part : data_parts) + for (const auto & part : data_parts) { if (part->modification_time < base) partition_parts_sum_diff[part->info.partition_id] += (base - part->modification_time); From 4fdd0c37a5b0866aead938dfe239e5f0b1afd166 Mon Sep 17 00:00:00 2001 From: fastio Date: Tue, 26 Jul 2022 15:05:50 +0800 Subject: [PATCH 005/188] fix test case format --- .../test_auto_optimize_partitions/test.py | 45 ++++++++++++------- 1 file changed, 29 insertions(+), 16 deletions(-) diff --git a/tests/integration/test_auto_optimize_partitions/test.py b/tests/integration/test_auto_optimize_partitions/test.py index f829a062a5f..138d1136696 100644 --- a/tests/integration/test_auto_optimize_partitions/test.py +++ b/tests/integration/test_auto_optimize_partitions/test.py @@ -8,7 +8,8 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", main_configs=["configs/zookeeper_config.xml", "configs/merge_tree.xml"], - with_zookeeper=True) + with_zookeeper=True, +) @pytest.fixture(scope="module") @@ -29,11 +30,15 @@ def test_without_auto_optimize_merge_tree(start_cluster): time.sleep(5) - expected = TSV('''3\n''') - assert TSV( - node.query( - "SELECT count(*) FROM system.parts where table='test' and active=1" - )) == expected + expected = TSV("""3\n""") + assert ( + TSV( + node.query( + "SELECT count(*) FROM system.parts where table='test' and active=1" + ) + ) + == expected + ) node.query("DROP TABLE test;") @@ -48,11 +53,15 @@ def test_auto_optimize_merge_tree(start_cluster): time.sleep(10) - expected = TSV('''1\n''') - assert TSV( - node.query( - "SELECT count(*) FROM system.parts where table='test' and active=1" - )) == expected + expected = TSV("""1\n""") + assert ( + TSV( + node.query( + "SELECT count(*) FROM system.parts where table='test' and active=1" + ) + ) + == expected + ) node.query("DROP TABLE test;") @@ -67,10 +76,14 @@ def test_auto_optimize_replicated_merge_tree(start_cluster): time.sleep(10) - expected = TSV('''1\n''') - assert TSV( - node.query( - "SELECT count(*) FROM system.parts where table='test' and active=1" - )) == expected + expected = TSV("""1\n""") + assert ( + TSV( + node.query( + "SELECT count(*) FROM system.parts where table='test' and active=1" + ) + ) + == expected + ) node.query("DROP TABLE test;") From e113d697bec7dda8a0b4d90ed21dacc6e28eb7f5 Mon Sep 17 00:00:00 2001 From: fastio Date: Mon, 1 Aug 2022 11:04:18 +0800 Subject: [PATCH 006/188] Update src/Storages/MergeTree/MergeTreeSettings.h Co-authored-by: Antonio Andelic --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 6b780a4d718..93fafa0f6c5 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -62,7 +62,7 @@ struct Settings; M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \ M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ M(UInt64, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds, 1ULL * 3600 * 24 * 30, "Remove old broken detached parts in the background if they remained intouched for a specified by this setting period of time.", 0) \ - M(UInt64, auto_optimize_partition_interval_seconds, 600, "The period of executing the auto optimize partitions in background.", 0) \ + M(UInt64, auto_optimize_partition_interval_seconds, 600, "The period of executing the auto optimize partitions in background. Set to 0 to disable.", 0) \ M(UInt64, merge_tree_enable_clear_old_broken_detached, false, "Enable clearing old broken detached parts operation in background.", 0) \ M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \ \ From e382406e0abf466983624d2e9fcacedcafffd902 Mon Sep 17 00:00:00 2001 From: fastio Date: Mon, 1 Aug 2022 11:04:48 +0800 Subject: [PATCH 007/188] Update src/Storages/StorageMergeTree.h Co-authored-by: Antonio Andelic --- src/Storages/StorageMergeTree.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index d5e4f978112..1b63c1d58cf 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -174,7 +174,7 @@ private: String * out_disable_reason = nullptr, bool optimize_skip_merged_partitions = false); - /** Determines what parts which within one partition should be merged. + /** Determines what part within one partition should be merged. */ std::shared_ptr selectOnePartitionToOptimize( const StorageMetadataPtr & metadata_snapshot, From cbe7b4e21b341425026f7f18e97c2772aed7123a Mon Sep 17 00:00:00 2001 From: fastio Date: Mon, 1 Aug 2022 11:58:19 +0800 Subject: [PATCH 008/188] Update src/Storages/StorageMergeTree.cpp Co-authored-by: Antonio Andelic --- src/Storages/StorageMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9b41e4dae53..02c756d2e57 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -814,10 +814,10 @@ std::shared_ptr StorageMergeTree::selectOnePartitionTo return nullptr; if (!merge_entry) { - constexpr const char * message = "Cannot OPTIMIZE table: {} in background"; + static constexpr const char * message = "Cannot OPTIMIZE table in background: {}"; if (disable_reason.empty()) disable_reason = "unknown reason"; - LOG_INFO(log, fmt::runtime(message), disable_reason); + LOG_INFO(log, message, disable_reason); } return merge_entry; } From eecccac2de58ca93e102fd111eadd423540ea2af Mon Sep 17 00:00:00 2001 From: fastio Date: Mon, 1 Aug 2022 17:07:30 +0800 Subject: [PATCH 009/188] refine codes and fix test cases --- src/Storages/StorageMergeTree.cpp | 15 ++---- .../test_auto_optimize_partitions/test.py | 46 +++++++------------ 2 files changed, 20 insertions(+), 41 deletions(-) mode change 100644 => 100755 src/Storages/StorageMergeTree.cpp diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp old mode 100644 new mode 100755 index 02c756d2e57..6db107afdc3 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -781,17 +781,8 @@ std::shared_ptr StorageMergeTree::selectOnePartitionTo if (part->modification_time < base) partition_parts_sum_diff[part->info.partition_id] += (base - part->modification_time); } - String best_partition_id; - Int32 max_diff = 0; - for (auto [partition_id, diff] : partition_parts_sum_diff) - { - if (diff > max_diff) - { - best_partition_id = partition_id; - max_diff = diff; - } - } - if (best_partition_id.empty()) + auto best_partition_it = std::max_element(partition_parts_sum_diff.begin(), partition_parts_sum_diff.end(), [](const auto & e1, const auto & e2) { return e1.second < e2.second; }); + if (best_partition_it == partition_parts_sum_diff.end()) { return nullptr; } @@ -801,7 +792,7 @@ std::shared_ptr StorageMergeTree::selectOnePartitionTo auto merge_entry = selectPartsToMerge( metadata_snapshot, true, - best_partition_id, + best_partition_it->first, true, &disable_reason, table_lock_holder, diff --git a/tests/integration/test_auto_optimize_partitions/test.py b/tests/integration/test_auto_optimize_partitions/test.py index 138d1136696..6b0ec076fbe 100644 --- a/tests/integration/test_auto_optimize_partitions/test.py +++ b/tests/integration/test_auto_optimize_partitions/test.py @@ -21,6 +21,20 @@ def start_cluster(): finally: cluster.shutdown() +def check_expected_result_or_fail(seconds, expected): + ok = False + for i in range(int(seconds) * 2): + result = TSV( + node.query( + "SELECT count(*) FROM system.parts where table='test' and active=1" + ) + ) + if result == expected: + ok = True + break + else: + time.sleep(0.5) + assert(ok) def test_without_auto_optimize_merge_tree(start_cluster): node.query("CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i;") @@ -28,17 +42,9 @@ def test_without_auto_optimize_merge_tree(start_cluster): node.query("INSERT INTO test SELECT 2") node.query("INSERT INTO test SELECT 3") - time.sleep(5) expected = TSV("""3\n""") - assert ( - TSV( - node.query( - "SELECT count(*) FROM system.parts where table='test' and active=1" - ) - ) - == expected - ) + check_expected_result_or_fail(5, expected) node.query("DROP TABLE test;") @@ -51,17 +57,8 @@ def test_auto_optimize_merge_tree(start_cluster): node.query("INSERT INTO test SELECT 2") node.query("INSERT INTO test SELECT 3") - time.sleep(10) - expected = TSV("""1\n""") - assert ( - TSV( - node.query( - "SELECT count(*) FROM system.parts where table='test' and active=1" - ) - ) - == expected - ) + check_expected_result_or_fail(10, expected) node.query("DROP TABLE test;") @@ -74,16 +71,7 @@ def test_auto_optimize_replicated_merge_tree(start_cluster): node.query("INSERT INTO test SELECT 2") node.query("INSERT INTO test SELECT 3") - time.sleep(10) - expected = TSV("""1\n""") - assert ( - TSV( - node.query( - "SELECT count(*) FROM system.parts where table='test' and active=1" - ) - ) - == expected - ) + check_expected_result_or_fail(10, expected) node.query("DROP TABLE test;") From d58df93e96ec8d911b83eec00474a04278898912 Mon Sep 17 00:00:00 2001 From: fastio Date: Mon, 1 Aug 2022 21:40:39 +0800 Subject: [PATCH 010/188] fix test case code style --- tests/integration/test_auto_optimize_partitions/test.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_auto_optimize_partitions/test.py b/tests/integration/test_auto_optimize_partitions/test.py index 6b0ec076fbe..dcd6d5031b2 100644 --- a/tests/integration/test_auto_optimize_partitions/test.py +++ b/tests/integration/test_auto_optimize_partitions/test.py @@ -21,6 +21,7 @@ def start_cluster(): finally: cluster.shutdown() + def check_expected_result_or_fail(seconds, expected): ok = False for i in range(int(seconds) * 2): @@ -34,7 +35,8 @@ def check_expected_result_or_fail(seconds, expected): break else: time.sleep(0.5) - assert(ok) + assert ok + def test_without_auto_optimize_merge_tree(start_cluster): node.query("CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i;") @@ -42,7 +44,6 @@ def test_without_auto_optimize_merge_tree(start_cluster): node.query("INSERT INTO test SELECT 2") node.query("INSERT INTO test SELECT 3") - expected = TSV("""3\n""") check_expected_result_or_fail(5, expected) From 93d88db7a5b409ce00840a888d6d14362e1bb157 Mon Sep 17 00:00:00 2001 From: fastio Date: Thu, 4 Aug 2022 11:00:02 +0800 Subject: [PATCH 011/188] fix wronge permission --- src/Storages/StorageMergeTree.cpp | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 src/Storages/StorageMergeTree.cpp diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp old mode 100755 new mode 100644 From d92b9e07fdc925ffde1106ce8131bbf3ded3ade3 Mon Sep 17 00:00:00 2001 From: fastio Date: Tue, 9 Aug 2022 09:32:57 +0800 Subject: [PATCH 012/188] Remove auto_optimize_partition_after_seconds from the obsolete --- src/Core/Settings.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 99ad00df70e..bda72f089eb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -634,7 +634,6 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) MAKE_OBSOLETE(M, Bool, allow_experimental_map_type, true) \ MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60) \ MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \ - MAKE_OBSOLETE(M, UInt64, auto_optimize_partition_interval_seconds, 600) \ MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \ MAKE_OBSOLETE(M, Bool, allow_experimental_projection_optimization, true) \ From 9cb74c7807964996e40091f7d6dff4e265a6eeb1 Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Fri, 23 Sep 2022 14:12:37 +0200 Subject: [PATCH 013/188] Low cardinality cases moved to the function for its corresponding type --- src/Functions/FunctionsJSON.cpp | 70 ++++++++++++------ .../performance/low_cardinality_from_json.xml | 73 +++++++++++++++++++ .../02452_check_low_cardinality.reference | 7 ++ .../02452_check_low_cardinality.sql | 61 ++++++++++++++++ 4 files changed, 190 insertions(+), 21 deletions(-) create mode 100644 tests/performance/low_cardinality_from_json.xml create mode 100644 tests/queries/0_stateless/02452_check_low_cardinality.reference create mode 100644 tests/queries/0_stateless/02452_check_low_cardinality.sql diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 814f709af27..95b3b11bc5b 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -694,9 +695,15 @@ public: } else return false; - - auto & col_vec = assert_cast &>(dest); - col_vec.insertValue(value); + + if (dest.getDataType() == TypeIndex::LowCardinality) { + ColumnLowCardinality & col_low = assert_cast(dest); + col_low.insertData(reinterpret_cast(&value), sizeof(value)); + } + else { + auto & col_vec = assert_cast &>(dest); + col_vec.insertValue(value); + } return true; } }; @@ -773,8 +780,15 @@ public: return JSONExtractRawImpl::insertResultToColumn(dest, element, {}); auto str = element.getString(); - ColumnString & col_str = assert_cast(dest); - col_str.insertData(str.data(), str.size()); + + if (dest.getDataType() == TypeIndex::LowCardinality) { + ColumnLowCardinality & col_low = assert_cast(dest); + col_low.insertData(str.data(), str.size()); + } + else { + ColumnString & col_str = assert_cast(dest); + col_str.insertData(str.data(), str.size()); + } return true; } }; @@ -803,25 +817,26 @@ struct JSONExtractTree } }; - class LowCardinalityNode : public Node + class LowCardinalityFixedStringNode : public Node { public: - LowCardinalityNode(DataTypePtr dictionary_type_, std::unique_ptr impl_) - : dictionary_type(dictionary_type_), impl(std::move(impl_)) {} + LowCardinalityFixedStringNode(const size_t fixed_length_) + : fixed_length(fixed_length_) {} bool insertResultToColumn(IColumn & dest, const Element & element) override { - auto from_col = dictionary_type->createColumn(); - if (impl->insertResultToColumn(*from_col, element)) - { - std::string_view value = from_col->getDataAt(0).toView(); - assert_cast(dest).insertData(value.data(), value.size()); - return true; - } - return false; + auto str = element.getString(); + + if (str.size() > fixed_length) + return false; + + auto padded_str = str.data() + std::string(fixed_length - std::min(fixed_length, str.length()), '\0'); + + assert_cast(dest).insertData(padded_str.data(), padded_str.size()); + return true; + } private: - DataTypePtr dictionary_type; - std::unique_ptr impl; + const size_t fixed_length; }; class UUIDNode : public Node @@ -833,7 +848,13 @@ struct JSONExtractTree return false; auto uuid = parseFromString(element.getString()); - assert_cast(dest).insert(uuid); + if (dest.getDataType() == TypeIndex::LowCardinality) { + ColumnLowCardinality & col_low = assert_cast(dest); + col_low.insertData(reinterpret_cast(&uuid), sizeof(uuid)); + } + else { + assert_cast(dest).insert(uuid); + } return true; } }; @@ -873,11 +894,12 @@ struct JSONExtractTree { if (!element.isString()) return false; - auto & col_str = assert_cast(dest); auto str = element.getString(); + auto & col_str = assert_cast(dest); if (str.size() > col_str.getN()) return false; col_str.insertData(str.data(), str.size()); + return true; } }; @@ -1101,7 +1123,13 @@ struct JSONExtractTree { auto dictionary_type = typeid_cast(type.get())->getDictionaryType(); auto impl = build(function_name, dictionary_type); - return std::make_unique(dictionary_type, std::move(impl)); + + if ((*dictionary_type).getTypeId() == TypeIndex::FixedString) { + auto fixed_length = typeid_cast(dictionary_type.get())->getN(); + return std::make_unique(fixed_length); + } + else + return impl; } case TypeIndex::Decimal256: return std::make_unique>(type); case TypeIndex::Decimal128: return std::make_unique>(type); diff --git a/tests/performance/low_cardinality_from_json.xml b/tests/performance/low_cardinality_from_json.xml new file mode 100644 index 00000000000..ac6542ac503 --- /dev/null +++ b/tests/performance/low_cardinality_from_json.xml @@ -0,0 +1,73 @@ + + + + + string_json + + '{"a": "hi", "b": "hello", "c": "hola", "d": "see you, bye, bye"}' + + + + int_json + + '{"a": 11, "b": 2222, "c": 33333333, "d": 4444444444444444}' + + + + uuid_json + + '{"a": "2d49dc6e-ddce-4cd0-afb8-790956df54c4", "b": "2d49dc6e-ddce-4cd0-afb8-790956df54c3", "c": "2d49dc6e-ddce-4cd0-afb8-790956df54c1", "d": "2d49dc6e-ddce-4cd0-afb8-790956df54c1"}' + + + + low_cardinality_tuple_string + + 'Tuple(a LowCardinality(String), b LowCardinality(String), c LowCardinality(String), d LowCardinality(String) )' + + + + low_cardinality_tuple_fixed_string + + 'Tuple(a LowCardinality(FixedString(20)), b LowCardinality(FixedString(20)), c LowCardinality(FixedString(20)), d LowCardinality(FixedString(20)) )' + + + + low_cardinality_tuple_int8 + + 'Tuple(a LowCardinality(Int8), b LowCardinality(Int8), c LowCardinality(Int8), d LowCardinality(Int8) )' + + + + low_cardinality_tuple_int16 + + 'Tuple(a LowCardinality(Int16), b LowCardinality(Int16), c LowCardinality(Int16), d LowCardinality(Int16) )' + + + + low_cardinality_tuple_int32 + + 'Tuple(a LowCardinality(Int32), b LowCardinality(Int32), c LowCardinality(Int32), d LowCardinality(Int32) )' + + + + low_cardinality_tuple_int64 + + 'Tuple(a LowCardinality(Int64), b LowCardinality(Int64), c LowCardinality(Int64), d LowCardinality(Int64) )' + + + + low_cardinality_tuple_uuid + + 'Tuple(a LowCardinality(UUID), b LowCardinality(UUID), c LowCardinality(UUID), d LowCardinality(UUID) )' + + + + + SELECT 'fixed_string_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({string_json}), {low_cardinality_tuple_fixed_string})) FORMAT Null + SELECT 'string_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({string_json}), {low_cardinality_tuple_string})) FORMAT Null + SELECT 'int8_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int8})) FORMAT Null + SELECT 'int16_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int16})) FORMAT Null + SELECT 'int32_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int32})) FORMAT Null + SELECT 'int64_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int64})) FORMAT Null + SELECT 'uuid_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({uuid_json}), {low_cardinality_tuple_uuid})) FORMAT Null + \ No newline at end of file diff --git a/tests/queries/0_stateless/02452_check_low_cardinality.reference b/tests/queries/0_stateless/02452_check_low_cardinality.reference new file mode 100644 index 00000000000..700778e02c7 --- /dev/null +++ b/tests/queries/0_stateless/02452_check_low_cardinality.reference @@ -0,0 +1,7 @@ +('hi','hello','hola','see you, bye, bye') +('hi\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','hello\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','hola\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','see you, bye, bye\0\0\0') +(11,0,0,0) +(11,2222,0,0) +(11,2222,33333333,0) +(11,2222,33333333,4444444444444444) +('2d49dc6e-ddce-4cd0-afb8-790956df54c4','2d49dc6e-ddce-4cd0-afb8-790956df54c3','2d49dc6e-ddce-4cd0-afb8-790956df54c1','2d49dc6e-ddce-4cd0-afb8-790956df54c1') diff --git a/tests/queries/0_stateless/02452_check_low_cardinality.sql b/tests/queries/0_stateless/02452_check_low_cardinality.sql new file mode 100644 index 00000000000..561acefa770 --- /dev/null +++ b/tests/queries/0_stateless/02452_check_low_cardinality.sql @@ -0,0 +1,61 @@ +DROP TABLE IF EXISTS test_low_cardinality_string; +DROP TABLE IF EXISTS test_low_cardinality_uuid; +DROP TABLE IF EXISTS test_low_cardinality_int; +CREATE TABLE test_low_cardinality_string (data String) ENGINE MergeTree ORDER BY data; +CREATE TABLE test_low_cardinality_uuid (data String) ENGINE MergeTree ORDER BY data; +CREATE TABLE test_low_cardinality_int (data String) ENGINE MergeTree ORDER BY data; +INSERT INTO test_low_cardinality_string (data) VALUES ('{"a": "hi", "b": "hello", "c": "hola", "d": "see you, bye, bye"}'); +INSERT INTO test_low_cardinality_int (data) VALUES ('{"a": 11, "b": 2222, "c": 33333333, "d": 4444444444444444}'); +INSERT INTO test_low_cardinality_uuid (data) VALUES ('{"a": "2d49dc6e-ddce-4cd0-afb8-790956df54c4", "b": "2d49dc6e-ddce-4cd0-afb8-790956df54c3", "c": "2d49dc6e-ddce-4cd0-afb8-790956df54c1", "d": "2d49dc6e-ddce-4cd0-afb8-790956df54c1"}'); +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(String), + b LowCardinality(String), + c LowCardinality(String), + d LowCardinality(String) + + )') AS json FROM test_low_cardinality_string; +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(FixedString(20)), + b LowCardinality(FixedString(20)), + c LowCardinality(FixedString(20)), + d LowCardinality(FixedString(20)) + + )') AS json FROM test_low_cardinality_string; +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(Int8), + b LowCardinality(Int8), + c LowCardinality(Int8), + d LowCardinality(Int8) + + )') AS json FROM test_low_cardinality_int; +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(Int16), + b LowCardinality(Int16), + c LowCardinality(Int16), + d LowCardinality(Int16) + + )') AS json FROM test_low_cardinality_int; +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(Int32), + b LowCardinality(Int32), + c LowCardinality(Int32), + d LowCardinality(Int32) + + )') AS json FROM test_low_cardinality_int; +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(Int64), + b LowCardinality(Int64), + c LowCardinality(Int64), + d LowCardinality(Int64) + + )') AS json FROM test_low_cardinality_int; +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(UUID), + b LowCardinality(UUID), + c LowCardinality(UUID), + d LowCardinality(UUID) + + )') AS json FROM test_low_cardinality_uuid; +DROP TABLE test_low_cardinality_string; +DROP TABLE test_low_cardinality_uuid; +DROP TABLE test_low_cardinality_int; From dd47d7fd66285397bb2259ce2bcd8d99414aa9f8 Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Fri, 23 Sep 2022 17:13:59 +0200 Subject: [PATCH 014/188] Fixed style --- src/Functions/FunctionsJSON.cpp | 36 +++++++++++++++++++-------------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 95b3b11bc5b..6ff0c319559 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -695,12 +695,14 @@ public: } else return false; - - if (dest.getDataType() == TypeIndex::LowCardinality) { + + if (dest.getDataType() == TypeIndex::LowCardinality) + { ColumnLowCardinality & col_low = assert_cast(dest); col_low.insertData(reinterpret_cast(&value), sizeof(value)); } - else { + else + { auto & col_vec = assert_cast &>(dest); col_vec.insertValue(value); } @@ -781,11 +783,13 @@ public: auto str = element.getString(); - if (dest.getDataType() == TypeIndex::LowCardinality) { + if (dest.getDataType() == TypeIndex::LowCardinality) + { ColumnLowCardinality & col_low = assert_cast(dest); col_low.insertData(str.data(), str.size()); } - else { + else + { ColumnString & col_str = assert_cast(dest); col_str.insertData(str.data(), str.size()); } @@ -825,7 +829,7 @@ struct JSONExtractTree bool insertResultToColumn(IColumn & dest, const Element & element) override { auto str = element.getString(); - + if (str.size() > fixed_length) return false; @@ -848,11 +852,13 @@ struct JSONExtractTree return false; auto uuid = parseFromString(element.getString()); - if (dest.getDataType() == TypeIndex::LowCardinality) { + if (dest.getDataType() == TypeIndex::LowCardinality) + { ColumnLowCardinality & col_low = assert_cast(dest); col_low.insertData(reinterpret_cast(&uuid), sizeof(uuid)); } - else { + else + { assert_cast(dest).insert(uuid); } return true; @@ -894,12 +900,13 @@ struct JSONExtractTree { if (!element.isString()) return false; + auto str = element.getString(); auto & col_str = assert_cast(dest); if (str.size() > col_str.getN()) return false; col_str.insertData(str.data(), str.size()); - + return true; } }; @@ -1122,14 +1129,13 @@ struct JSONExtractTree case TypeIndex::LowCardinality: { auto dictionary_type = typeid_cast(type.get())->getDictionaryType(); - auto impl = build(function_name, dictionary_type); - - if ((*dictionary_type).getTypeId() == TypeIndex::FixedString) { + if ((*dictionary_type).getTypeId() == TypeIndex::FixedString) + { auto fixed_length = typeid_cast(dictionary_type.get())->getN(); - return std::make_unique(fixed_length); + return std::make_unique(fixed_length); } - else - return impl; + auto impl = build(function_name, dictionary_type); + return impl; } case TypeIndex::Decimal256: return std::make_unique>(type); case TypeIndex::Decimal128: return std::make_unique>(type); From 48b646aece7815e1d5a8b9eaa6e369d6264b57cb Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Mon, 26 Sep 2022 14:55:54 +0200 Subject: [PATCH 015/188] Added no-fasttest flag in test 02452_check_low_cardinality --- tests/queries/0_stateless/02452_check_low_cardinality.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02452_check_low_cardinality.sql b/tests/queries/0_stateless/02452_check_low_cardinality.sql index 561acefa770..435f48bb16a 100644 --- a/tests/queries/0_stateless/02452_check_low_cardinality.sql +++ b/tests/queries/0_stateless/02452_check_low_cardinality.sql @@ -1,3 +1,4 @@ +-- Tags: no-fasttest DROP TABLE IF EXISTS test_low_cardinality_string; DROP TABLE IF EXISTS test_low_cardinality_uuid; DROP TABLE IF EXISTS test_low_cardinality_int; From b4f481e7245eaec32d197df6571778ca1f61d90e Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Tue, 27 Sep 2022 10:20:54 +0200 Subject: [PATCH 016/188] Added some comments and fixed some format --- src/Functions/FunctionsJSON.cpp | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 6ff0c319559..d2bed55cd52 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -699,7 +699,7 @@ public: if (dest.getDataType() == TypeIndex::LowCardinality) { ColumnLowCardinality & col_low = assert_cast(dest); - col_low.insertData(reinterpret_cast(&value), sizeof(value)); + col_low.insertData(reinterpret_cast(&value), sizeof(value)); } else { @@ -824,8 +824,7 @@ struct JSONExtractTree class LowCardinalityFixedStringNode : public Node { public: - LowCardinalityFixedStringNode(const size_t fixed_length_) - : fixed_length(fixed_length_) {} + LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) {} bool insertResultToColumn(IColumn & dest, const Element & element) override { auto str = element.getString(); @@ -833,12 +832,15 @@ struct JSONExtractTree if (str.size() > fixed_length) return false; + // For the non low cardinality case of FixedString, the padding is done in the FixedString Column implementation. + // In order to avoid having to pass the data to a FixedString Column and read it back (which would slow down the execution) + // the data is padded here and writen directly to the Low Cardinality Column auto padded_str = str.data() + std::string(fixed_length - std::min(fixed_length, str.length()), '\0'); assert_cast(dest).insertData(padded_str.data(), padded_str.size()); return true; - } + private: const size_t fixed_length; }; @@ -855,7 +857,7 @@ struct JSONExtractTree if (dest.getDataType() == TypeIndex::LowCardinality) { ColumnLowCardinality & col_low = assert_cast(dest); - col_low.insertData(reinterpret_cast(&uuid), sizeof(uuid)); + col_low.insertData(reinterpret_cast(&uuid), sizeof(uuid)); } else { @@ -880,6 +882,7 @@ struct JSONExtractTree assert_cast &>(dest).insert(result); return true; } + private: DataTypePtr data_type; }; @@ -1128,6 +1131,11 @@ struct JSONExtractTree case TypeIndex::UUID: return std::make_unique(); case TypeIndex::LowCardinality: { + // The low cardinality case is treated in two different ways: + // For FixedString type, an especial class is implemented for inserting the data in the destination column, + // as the string length must be passed in order to check and pad the incoming data. + // For the rest of low cardinality types, the insertion is done in their corresponding class, adapting the data + // as needed for the insertData function of the ColumnLowCardinality. auto dictionary_type = typeid_cast(type.get())->getDictionaryType(); if ((*dictionary_type).getTypeId() == TypeIndex::FixedString) { From 8307b034a55d0f195912ffd11909f7cbd41bc344 Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Tue, 27 Sep 2022 10:56:52 +0200 Subject: [PATCH 017/188] Fixed typo --- src/Functions/FunctionsJSON.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index d2bed55cd52..336543ab3a5 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -834,7 +834,7 @@ struct JSONExtractTree // For the non low cardinality case of FixedString, the padding is done in the FixedString Column implementation. // In order to avoid having to pass the data to a FixedString Column and read it back (which would slow down the execution) - // the data is padded here and writen directly to the Low Cardinality Column + // the data is padded here and written directly to the Low Cardinality Column auto padded_str = str.data() + std::string(fixed_length - std::min(fixed_length, str.length()), '\0'); assert_cast(dest).insertData(padded_str.data(), padded_str.size()); From dc75c31855c12ffa01f7da6ecaa9a7c77c8cb7c2 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 28 Sep 2022 12:16:20 +0200 Subject: [PATCH 018/188] Update src/Functions/FunctionsJSON.cpp --- src/Functions/FunctionsJSON.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 336543ab3a5..371f4be12d5 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -824,7 +824,7 @@ struct JSONExtractTree class LowCardinalityFixedStringNode : public Node { public: - LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) {} + explicit LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) {} bool insertResultToColumn(IColumn & dest, const Element & element) override { auto str = element.getString(); From 875145a813161f1fa34c74de5c0536981a0c24b7 Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Fri, 30 Sep 2022 12:41:41 +0200 Subject: [PATCH 019/188] FIxed corner case in lowCardinalityFixedString and FixedString --- src/Functions/FunctionsJSON.cpp | 43 ++++++++++++++++++- .../02452_check_low_cardinality.reference | 1 + .../02452_check_low_cardinality.sql | 11 +++++ ...ct_fixed_string_from_nested_json.reference | 1 + ..._extract_fixed_string_from_nested_json.sql | 6 +++ 5 files changed, 60 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.reference create mode 100644 tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.sql diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 371f4be12d5..cb8debbc1d8 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -827,8 +827,13 @@ struct JSONExtractTree explicit LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) {} bool insertResultToColumn(IColumn & dest, const Element & element) override { - auto str = element.getString(); + // If element is an object we delegate the insertion to JSONExtractRawImpl + if (element.isObject()) + return JSONExtractRawImpl::insertResultToLowCardinalityFixedStringColumn(dest, element, fixed_length); + else if (!element.isString()) + return false; + auto str = element.getString(); if (str.size() > fixed_length) return false; @@ -901,9 +906,12 @@ struct JSONExtractTree public: bool insertResultToColumn(IColumn & dest, const Element & element) override { - if (!element.isString()) + if (element.isNull()) return false; + if (!element.isString()) + return JSONExtractRawImpl::insertResultToFixedStringColumn(dest, element, {}); + auto str = element.getString(); auto & col_str = assert_cast(dest); if (str.size() > col_str.getN()) @@ -1305,6 +1313,37 @@ public: return true; } + // We use insertResultToFixedStringColumn in case we are inserting raw data in a FixedString column + static bool insertResultToFixedStringColumn(IColumn & dest, const Element & element, std::string_view) + { + ColumnFixedString & col_str = assert_cast(dest); + auto & chars = col_str.getChars(); + WriteBufferFromVector buf(chars, AppendModeTag()); + traverse(element, buf); + buf.finalize(); + col_str.insertDefault(); + return true; + } + + // We use insertResultToLowCardinalityFixedStringColumn in case we are inserting raw data in a Low Cardinality FixedString column + static bool insertResultToLowCardinalityFixedStringColumn(IColumn & dest, const Element & element, size_t fixed_length) + { + if (element.getObject().size() > fixed_length) + return false; + + ColumnFixedString::Chars chars; + WriteBufferFromVector buf(chars, AppendModeTag()); + traverse(element, buf); + buf.finalize(); + chars.push_back(0); + std::string str = reinterpret_cast(chars.data()); + + auto padded_str = str.data() + std::string(fixed_length - std::min(fixed_length, str.length()), '\0'); + assert_cast(dest).insertData(padded_str.data(), padded_str.size()); + + return true; + } + private: static void traverse(const Element & element, WriteBuffer & buf) { diff --git a/tests/queries/0_stateless/02452_check_low_cardinality.reference b/tests/queries/0_stateless/02452_check_low_cardinality.reference index 700778e02c7..dc5553d7886 100644 --- a/tests/queries/0_stateless/02452_check_low_cardinality.reference +++ b/tests/queries/0_stateless/02452_check_low_cardinality.reference @@ -5,3 +5,4 @@ (11,2222,33333333,0) (11,2222,33333333,4444444444444444) ('2d49dc6e-ddce-4cd0-afb8-790956df54c4','2d49dc6e-ddce-4cd0-afb8-790956df54c3','2d49dc6e-ddce-4cd0-afb8-790956df54c1','2d49dc6e-ddce-4cd0-afb8-790956df54c1') +('{"b":{"c":1,"d":"str"}}\0\0\0\0\0\0\0','\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0') diff --git a/tests/queries/0_stateless/02452_check_low_cardinality.sql b/tests/queries/0_stateless/02452_check_low_cardinality.sql index 435f48bb16a..d62f418139e 100644 --- a/tests/queries/0_stateless/02452_check_low_cardinality.sql +++ b/tests/queries/0_stateless/02452_check_low_cardinality.sql @@ -1,11 +1,14 @@ -- Tags: no-fasttest DROP TABLE IF EXISTS test_low_cardinality_string; +DROP TABLE IF EXISTS test_low_cardinality_string_nested_json; DROP TABLE IF EXISTS test_low_cardinality_uuid; DROP TABLE IF EXISTS test_low_cardinality_int; CREATE TABLE test_low_cardinality_string (data String) ENGINE MergeTree ORDER BY data; +CREATE TABLE test_low_cardinality_string_nested_json (data String) ENGINE MergeTree ORDER BY data; CREATE TABLE test_low_cardinality_uuid (data String) ENGINE MergeTree ORDER BY data; CREATE TABLE test_low_cardinality_int (data String) ENGINE MergeTree ORDER BY data; INSERT INTO test_low_cardinality_string (data) VALUES ('{"a": "hi", "b": "hello", "c": "hola", "d": "see you, bye, bye"}'); +INSERT INTO test_low_cardinality_string_nested_json (data) VALUES ('{"a" : {"b" : {"c" : 1, "d" : "str"}}}'); INSERT INTO test_low_cardinality_int (data) VALUES ('{"a": 11, "b": 2222, "c": 33333333, "d": 4444444444444444}'); INSERT INTO test_low_cardinality_uuid (data) VALUES ('{"a": "2d49dc6e-ddce-4cd0-afb8-790956df54c4", "b": "2d49dc6e-ddce-4cd0-afb8-790956df54c3", "c": "2d49dc6e-ddce-4cd0-afb8-790956df54c1", "d": "2d49dc6e-ddce-4cd0-afb8-790956df54c1"}'); SELECT JSONExtract(data, 'Tuple( @@ -57,6 +60,14 @@ SELECT JSONExtract(data, 'Tuple( d LowCardinality(UUID) )') AS json FROM test_low_cardinality_uuid; +SELECT JSONExtract('{"a" : {"b" : {"c" : 1, "d" : "str"}}}', 'Tuple( + a LowCardinality(FixedString(30)), + b LowCardinality(FixedString(20)), + c LowCardinality(FixedString(20)), + d LowCardinality(FixedString(20)) + + )') AS json FROM test_low_cardinality_string; +DROP TABLE test_low_cardinality_string_nested_json; DROP TABLE test_low_cardinality_string; DROP TABLE test_low_cardinality_uuid; DROP TABLE test_low_cardinality_int; diff --git a/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.reference b/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.reference new file mode 100644 index 00000000000..3a528a24821 --- /dev/null +++ b/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.reference @@ -0,0 +1 @@ +('{"b":{"c":1,"d":"str"}}\0') diff --git a/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.sql b/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.sql new file mode 100644 index 00000000000..c3eee60f161 --- /dev/null +++ b/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.sql @@ -0,0 +1,6 @@ +-- Tags: no-fasttest +DROP TABLE IF EXISTS test_fixed_string_nested_json; +CREATE TABLE test_fixed_string_nested_json (data String) ENGINE MergeTree ORDER BY data; +INSERT INTO test_fixed_string_nested_json (data) VALUES ('{"a" : {"b" : {"c" : 1, "d" : "str"}}}'); +SELECT JSONExtract(data, 'Tuple(a FixedString(24))') AS json FROM test_fixed_string_nested_json; +DROP TABLE test_fixed_string_nested_json; From b1fced367eb16eb834507a2806cccfc14273333a Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Mon, 3 Oct 2022 09:59:52 +0200 Subject: [PATCH 020/188] Fixed warning --- src/Functions/FunctionsJSON.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index cb8debbc1d8..8968f302f58 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -1338,7 +1338,7 @@ public: chars.push_back(0); std::string str = reinterpret_cast(chars.data()); - auto padded_str = str.data() + std::string(fixed_length - std::min(fixed_length, str.length()), '\0'); + auto padded_str = str + std::string(fixed_length - std::min(fixed_length, str.length()), '\0'); assert_cast(dest).insertData(padded_str.data(), padded_str.size()); return true; From 06b2eeb5a74df9ceef28aee9e8a0b1c364e4256b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 18 Oct 2022 09:31:16 +0000 Subject: [PATCH 021/188] Small changes to settings --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 5 +++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index b078643afc0..65c0780a76b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -63,6 +63,7 @@ struct Settings; M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ M(UInt64, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds, 1ULL * 3600 * 24 * 30, "Remove old broken detached parts in the background if they remained intouched for a specified by this setting period of time.", 0) \ M(UInt64, auto_optimize_partition_interval_seconds, 600, "The period of executing the auto optimize partitions in background. Set to 0 to disable.", 0) \ + M(UInt64, auto_optimize_partition_after_seconds, 0, "The number of seconds since last mutation required for partitions to be automatically optimized. Set to 0 to disable.", 0) \ M(UInt64, merge_tree_enable_clear_old_broken_detached, false, "Enable clearing old broken detached parts operation in background.", 0) \ M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \ \ @@ -164,7 +165,6 @@ struct Settings; M(UInt64, replicated_max_parallel_fetches, 0, "Obsolete setting, does nothing.", 0) \ M(UInt64, replicated_max_parallel_fetches_for_table, 0, "Obsolete setting, does nothing.", 0) \ M(Bool, write_final_mark, true, "Obsolete setting, does nothing.", 0) \ - M(UInt64, auto_optimize_partition_after_seconds, 0, "The number seconds of which to do automatically optimization for partitions. Zero means disable.", 0) /// Settings that should not change after the creation of a table. /// NOLINTNEXTLINE #define APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(M) \ diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ff1d31c4570..93aac82565e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1203,7 +1203,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign if (!merge_entry) mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr, share_lock, lock); - if (getSettings()->auto_optimize_partition_after_seconds + if (getSettings()->auto_optimize_partition_interval_seconds && !merge_entry && !mutate_entry && time_after_previous_optimize_one_partition.compareAndRestartDeferred(getSettings()->auto_optimize_partition_interval_seconds)) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 37ba75813a8..bfc0d0520ff 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -293,7 +293,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( merge_selecting_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mergeSelectingTask)", [this] { mergeSelectingTask(); }); - if (getSettings()->auto_optimize_partition_after_seconds) + if (getSettings()->auto_optimize_partition_interval_seconds) auto_optimize_partition_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::autoOptimizePartitionTask)", [this] { autoOptimizePartitionTask(); }); @@ -4459,8 +4459,9 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con void StorageReplicatedMergeTree::autoOptimizePartitionTask() { - if (!is_leader || !getSettings()->auto_optimize_partition_after_seconds) + if (!is_leader) return; + auto table_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); try { From e828dce3311c16283a9dfe7dbd9a0332875b8d1a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 18 Oct 2022 12:11:23 +0000 Subject: [PATCH 022/188] Disable auto optimize by default --- src/Storages/MergeTree/MergeTreeSettings.h | 4 ++-- tests/integration/test_auto_optimize_partitions/test.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 65c0780a76b..63421c3de5d 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -62,8 +62,8 @@ struct Settings; M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \ M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ M(UInt64, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds, 1ULL * 3600 * 24 * 30, "Remove old broken detached parts in the background if they remained intouched for a specified by this setting period of time.", 0) \ - M(UInt64, auto_optimize_partition_interval_seconds, 600, "The period of executing the auto optimize partitions in background. Set to 0 to disable.", 0) \ - M(UInt64, auto_optimize_partition_after_seconds, 0, "The number of seconds since last mutation required for partitions to be automatically optimized. Set to 0 to disable.", 0) \ + M(UInt64, auto_optimize_partition_interval_seconds, 0, "The period of executing the auto optimize partitions in background. Set to 0 to disable.", 0) \ + M(UInt64, auto_optimize_partition_after_seconds, 0, "The number of seconds since last mutation required for partition to be automatically optimized. Set to 0 to disable.", 0) \ M(UInt64, merge_tree_enable_clear_old_broken_detached, false, "Enable clearing old broken detached parts operation in background.", 0) \ M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \ \ diff --git a/tests/integration/test_auto_optimize_partitions/test.py b/tests/integration/test_auto_optimize_partitions/test.py index dcd6d5031b2..77ce56e3787 100644 --- a/tests/integration/test_auto_optimize_partitions/test.py +++ b/tests/integration/test_auto_optimize_partitions/test.py @@ -52,7 +52,7 @@ def test_without_auto_optimize_merge_tree(start_cluster): def test_auto_optimize_merge_tree(start_cluster): node.query( - "CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i SETTINGS auto_optimize_partition_after_seconds=5;" + "CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i SETTINGS auto_optimize_partition_interval_seconds=5;" ) node.query("INSERT INTO test SELECT 1") node.query("INSERT INTO test SELECT 2") @@ -66,7 +66,7 @@ def test_auto_optimize_merge_tree(start_cluster): def test_auto_optimize_replicated_merge_tree(start_cluster): node.query( - "CREATE TABLE test (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') ORDER BY i SETTINGS auto_optimize_partition_after_seconds=5;" + "CREATE TABLE test (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') ORDER BY i SETTINGS auto_optimize_partition_interval_seconds=5;" ) node.query("INSERT INTO test SELECT 1") node.query("INSERT INTO test SELECT 2") From b125fd87189d490cbb9a9101e0a49006a97d534a Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Tue, 18 Oct 2022 09:34:53 -0400 Subject: [PATCH 023/188] Update uk-price-paid.md --- docs/en/getting-started/example-datasets/uk-price-paid.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index ef20c03883f..2a89bfda2e7 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -101,7 +101,7 @@ SELECT count() FROM uk_price_paid ``` -At the time this query was executed, the dataset had 27,450,499 rows. Let's see what the storage size is of the table in ClickHouse: +At the time this query was run, the dataset had 27,450,499 rows. Let's see what the storage size is of the table in ClickHouse: ```sql SELECT formatReadableSize(total_bytes) @@ -342,7 +342,7 @@ The result looks like: ## Let's Speed Up Queries Using Projections {#speedup-with-projections} -[Projections](../../sql-reference/statements/alter/projection.md) allow you to improve query speeds by storing pre-aggregated data in whatever format you want. In this example, we create a projection that keeps track of the average price, total price, and count of properties grouped by the year, district and town. At execution time, ClickHouse will use your projection if it thinks the projection can improve the performance fo the query (you don't have to do anything special to use the projection - ClickHouse decides for you when the projection will be useful). +[Projections](../../sql-reference/statements/alter/projection.md) allow you to improve query speeds by storing pre-aggregated data in whatever format you want. In this example, we create a projection that keeps track of the average price, total price, and count of properties grouped by the year, district and town. At query time, ClickHouse will use your projection if it thinks the projection can improve the performance of the query (you don't have to do anything special to use the projection - ClickHouse decides for you when the projection will be useful). ### Build a Projection {#build-projection} From af28937cac643b797f38692f42a08d4ea3132c34 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 16 Oct 2022 22:09:20 +0200 Subject: [PATCH 024/188] packages: do not use type=config with directories archlinux packager does not support this yet. Signed-off-by: Azat Khuzhin --- packages/clickhouse-keeper.yaml | 4 ++-- packages/clickhouse-server.yaml | 7 +++++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/packages/clickhouse-keeper.yaml b/packages/clickhouse-keeper.yaml index 7803729c469..8f319c97b65 100644 --- a/packages/clickhouse-keeper.yaml +++ b/packages/clickhouse-keeper.yaml @@ -27,8 +27,8 @@ deb: Source: clickhouse contents: -- src: root/etc/clickhouse-keeper - dst: /etc/clickhouse-keeper +- src: root/etc/clickhouse-keeper/keeper_config.xml + dst: /etc/clickhouse-keeper/keeper_config.xml type: config - src: root/usr/bin/clickhouse-keeper dst: /usr/bin/clickhouse-keeper diff --git a/packages/clickhouse-server.yaml b/packages/clickhouse-server.yaml index a94ad1e9169..b0778e6bf72 100644 --- a/packages/clickhouse-server.yaml +++ b/packages/clickhouse-server.yaml @@ -42,8 +42,11 @@ deb: Source: clickhouse contents: -- src: root/etc/clickhouse-server - dst: /etc/clickhouse-server +- src: root/etc/clickhouse-server/config.xml + dst: /etc/clickhouse-server/config.xml + type: config +- src: root/etc/clickhouse-server/users.xml + dst: /etc/clickhouse-server/users.xml type: config - src: clickhouse-server.init dst: /etc/init.d/clickhouse-server From ff146211a5e1f95a3a0395a001d61a091d4751fc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 16 Oct 2022 22:13:45 +0200 Subject: [PATCH 025/188] Do not build deb packages by default CI should be fine, since it set MAKE_DEB, see next commit. Signed-off-by: Azat Khuzhin --- packages/build | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/packages/build b/packages/build index c5ebf8641a3..53f01e1c223 100755 --- a/packages/build +++ b/packages/build @@ -28,6 +28,7 @@ HELP="${0} [--test] [--rpm] [-h|--help] --apk - build APK packages --rpm - build RPM packages --tgz - build tarball package + --deb - build deb package --help - show this help and exit Used envs: @@ -47,13 +48,15 @@ fi export CLICKHOUSE_VERSION_STRING - while [[ $1 == --* ]] do case "$1" in --test ) VERSION_POSTFIX+='+test' shift ;; + --deb ) + MAKE_DEB=1 + shift ;; --apk ) MAKE_APK=1 shift ;; @@ -131,13 +134,15 @@ CLICKHOUSE_VERSION_STRING+=$VERSION_POSTFIX echo -e "\nCurrent version is $CLICKHOUSE_VERSION_STRING" for config in clickhouse*.yaml; do - echo "Building deb package for $config" + if [ -n "$MAKE_DEB" ] || [ -n "$MAKE_TGZ" ]; then + echo "Building deb package for $config" - # Preserve package path - exec 9>&1 - PKG_PATH=$(nfpm package --target "$OUTPUT_DIR" --config "$config" --packager deb | tee /dev/fd/9) - PKG_PATH=${PKG_PATH##*created package: } - exec 9>&- + # Preserve package path + exec 9>&1 + PKG_PATH=$(nfpm package --target "$OUTPUT_DIR" --config "$config" --packager deb | tee /dev/fd/9) + PKG_PATH=${PKG_PATH##*created package: } + exec 9>&- + fi if [ -n "$MAKE_APK" ]; then echo "Building apk package for $config" From fb6474c35b7cd12689d76da518dc4b3adae73928 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 16 Oct 2022 22:17:06 +0200 Subject: [PATCH 026/188] Add a comment about MAKE_* env vars Signed-off-by: Azat Khuzhin --- docker/packager/packager | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/packager/packager b/docker/packager/packager index b4aa4ebdd91..83629dc7408 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -208,6 +208,7 @@ def parse_env_variables( cxx = cc.replace("gcc", "g++").replace("clang", "clang++") if package_type == "deb": + # NOTE: This are the env for packages/build script result.append("MAKE_DEB=true") cmake_flags.append("-DENABLE_TESTS=0") cmake_flags.append("-DENABLE_UTILS=0") @@ -268,6 +269,7 @@ def parse_env_variables( result.append('DISTCC_HOSTS="localhost/`nproc`"') if additional_pkgs: + # NOTE: This are the env for packages/build script result.append("MAKE_APK=true") result.append("MAKE_RPM=true") result.append("MAKE_TGZ=true") From 043c4163f4dc2865ad727a4501b8bb7c1b2305c5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 16 Oct 2022 21:39:45 +0200 Subject: [PATCH 027/188] Use nfpm packager for archlinux packages Since nfpm v2.20.0 it can build packages for archlinux [1]. [1]: https://github.com/goreleaser/nfpm/pull/543 Something like this: $ mkdir -p /src/packages/root $ cmake -DCMAKE_INSTALL_PREFIX=/usr /src $ DESTDIR=/src/packages/root ninja install $ cd /src/packages $ ./build --archlinux $ root@s3:/src/packages# ./build --archlinux Current version is 22.10.1.1 Building archlinux package for clickhouse-client.yaml using archlinux packager... created package: /src/clickhouse-client-22.10.1.1-1-x86_64.pkg.tar.zst ... Signed-off-by: Azat Khuzhin --- docker/packager/binary/Dockerfile | 2 +- packages/build | 8 +++++++ utils/CMakeLists.txt | 2 -- utils/package/CMakeLists.txt | 1 - utils/package/arch/CMakeLists.txt | 2 -- utils/package/arch/PKGBUILD.in | 33 -------------------------- utils/package/arch/README.md | 17 ------------- utils/package/arch/clickhouse.sysusers | 3 --- utils/package/arch/clickhouse.tmpfiles | 1 - utils/package/arch/logging.xml | 6 ----- 10 files changed, 9 insertions(+), 66 deletions(-) delete mode 100644 utils/package/CMakeLists.txt delete mode 100644 utils/package/arch/CMakeLists.txt delete mode 100644 utils/package/arch/PKGBUILD.in delete mode 100644 utils/package/arch/README.md delete mode 100644 utils/package/arch/clickhouse.sysusers delete mode 100644 utils/package/arch/clickhouse.tmpfiles delete mode 100644 utils/package/arch/logging.xml diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 2954cd574d0..77afc3e924b 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -73,7 +73,7 @@ RUN apt-get install binutils-riscv64-linux-gnu # Architecture of the image when BuildKit/buildx is used ARG TARGETARCH -ARG NFPM_VERSION=2.18.1 +ARG NFPM_VERSION=2.20.0 RUN arch=${TARGETARCH:-amd64} \ && curl -Lo /tmp/nfpm.deb "https://github.com/goreleaser/nfpm/releases/download/v${NFPM_VERSION}/nfpm_${arch}.deb" \ diff --git a/packages/build b/packages/build index 53f01e1c223..531e068338d 100755 --- a/packages/build +++ b/packages/build @@ -26,6 +26,7 @@ SOURCE=${SOURCE:-$PKG_ROOT} HELP="${0} [--test] [--rpm] [-h|--help] --test - adds '+test' prefix to version --apk - build APK packages + --archlinux - build archlinux packages --rpm - build RPM packages --tgz - build tarball package --deb - build deb package @@ -60,6 +61,9 @@ do --apk ) MAKE_APK=1 shift ;; + --archlinux ) + MAKE_ARCHLINUX=1 + shift ;; --rpm ) MAKE_RPM=1 shift ;; @@ -148,6 +152,10 @@ for config in clickhouse*.yaml; do echo "Building apk package for $config" nfpm package --target "$OUTPUT_DIR" --config "$config" --packager apk fi + if [ -n "$MAKE_ARCHLINUX" ]; then + echo "Building archlinux package for $config" + nfpm package --target "$OUTPUT_DIR" --config "$config" --packager archlinux + fi if [ -n "$MAKE_RPM" ]; then echo "Building rpm package for $config" nfpm package --target "$OUTPUT_DIR" --config "$config" --packager rpm diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index a79982bbd61..92a97a9c60e 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -44,5 +44,3 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (memcpy-bench) endif () endif () - -add_subdirectory (package) diff --git a/utils/package/CMakeLists.txt b/utils/package/CMakeLists.txt deleted file mode 100644 index 8c8a09adc0f..00000000000 --- a/utils/package/CMakeLists.txt +++ /dev/null @@ -1 +0,0 @@ -add_subdirectory (arch) diff --git a/utils/package/arch/CMakeLists.txt b/utils/package/arch/CMakeLists.txt deleted file mode 100644 index 4ee754fec56..00000000000 --- a/utils/package/arch/CMakeLists.txt +++ /dev/null @@ -1,2 +0,0 @@ -include ("${ClickHouse_SOURCE_DIR}/cmake/version.cmake") -configure_file (PKGBUILD.in PKGBUILD) diff --git a/utils/package/arch/PKGBUILD.in b/utils/package/arch/PKGBUILD.in deleted file mode 100644 index 4e068e8b8a2..00000000000 --- a/utils/package/arch/PKGBUILD.in +++ /dev/null @@ -1,33 +0,0 @@ -pkgname=clickhouse -pkgver=${VERSION_STRING} -pkgrel=1 -pkgdesc='An open-source column-oriented database management system that allows generating analytical data reports in real time' -arch=('x86_64') -url='https://clickhouse.com/' -license=('Apache') - -package() { - install -dm 755 $pkgdir/usr/lib/tmpfiles.d - install -dm 755 $pkgdir/usr/lib/sysusers.d - install -Dm 644 ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse.tmpfiles $pkgdir/usr/lib/tmpfiles.d/clickhouse.conf - install -Dm 644 ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse.sysusers $pkgdir/usr/lib/sysusers.d/clickhouse.conf - install -dm 755 $pkgdir/etc/clickhouse-server/config.d - install -Dm 644 ${CMAKE_CURRENT_SOURCE_DIR}/logging.xml $pkgdir/etc/clickhouse-server/config.d/logging.xml - # This code was requisited from kmeaw@ https://aur.archlinux.org/packages/clickhouse/ . - SRC=${ClickHouse_SOURCE_DIR} - BIN=${ClickHouse_BINARY_DIR} - mkdir -p $pkgdir/etc/clickhouse-server/ $pkgdir/etc/clickhouse-client/ - mkdir -p $pkgdir/usr/bin/ - mkdir -p $pkgdir/usr/lib/systemd/system - ln -s clickhouse-client $pkgdir/usr/bin/clickhouse-server - cp $SRC/programs/server/config.xml $SRC/programs/server/users.xml $pkgdir/etc/clickhouse-server/ - cp $BIN/programs/clickhouse $pkgdir/usr/bin/clickhouse-client - patchelf --remove-rpath $pkgdir/usr/bin/clickhouse-client - patchelf --replace-needed libz.so.1 libz-ng.so.1 $pkgdir/usr/bin/clickhouse-client - cp $SRC/programs/client/clickhouse-client.xml $pkgdir/etc/clickhouse-client/config.xml - compiler="libclickhouse-compiler.so" - if ! pacman -Q clang | grep '^clang 7'; then - compiler="" - fi - cp $SRC/debian/clickhouse-server.service $pkgdir/usr/lib/systemd/system -} diff --git a/utils/package/arch/README.md b/utils/package/arch/README.md deleted file mode 100644 index 0db5aac8080..00000000000 --- a/utils/package/arch/README.md +++ /dev/null @@ -1,17 +0,0 @@ -### Build Arch Linux package - -From binary directory: - -``` -make -cd utils/package/arch -makepkg -``` - -### Install and start ClickHouse server - -``` -pacman -U clickhouse-*.pkg.tar.xz -systemctl enable clickhouse-server -systemctl start clickhouse-server -``` diff --git a/utils/package/arch/clickhouse.sysusers b/utils/package/arch/clickhouse.sysusers deleted file mode 100644 index 4381c52c4f2..00000000000 --- a/utils/package/arch/clickhouse.sysusers +++ /dev/null @@ -1,3 +0,0 @@ -u clickhouse - "ClickHouse user" /nonexistent /bin/false -g clickhouse - "ClickHouse group" -m clickhouse clickhouse diff --git a/utils/package/arch/clickhouse.tmpfiles b/utils/package/arch/clickhouse.tmpfiles deleted file mode 100644 index 631aa895f2f..00000000000 --- a/utils/package/arch/clickhouse.tmpfiles +++ /dev/null @@ -1 +0,0 @@ -d /var/lib/clickhouse 0700 clickhouse clickhouse diff --git a/utils/package/arch/logging.xml b/utils/package/arch/logging.xml deleted file mode 100644 index c7a78442424..00000000000 --- a/utils/package/arch/logging.xml +++ /dev/null @@ -1,6 +0,0 @@ - - - - - - From 5016597e5887e50e3237d7aad7ee6495ae783d49 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 16 Oct 2022 22:48:17 +0200 Subject: [PATCH 028/188] Add arch packages to .gitignore Signed-off-by: Azat Khuzhin --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index dd632eba85d..5b8f2ca452d 100644 --- a/.gitignore +++ b/.gitignore @@ -80,6 +80,7 @@ core vgcore* *.deb +*.tar.zst *.build *.upload *.changes From dbe68ab0a82c32142cff24f3c4a1920bbe902851 Mon Sep 17 00:00:00 2001 From: Quanfa Fu Date: Wed, 12 Oct 2022 22:54:16 +0800 Subject: [PATCH 029/188] Fix wrong behave of domain func with URLs contains UserInfo part and '@' When UserInfo part and '@' appear in the URL, the host after @ should be returned. For example, when url is "https://user:pass@clickhouse.com/", start_of_host should be char 'c' after '@', end_of_host should be '/' other than ':'. --- src/Functions/URL/domain.h | 26 +++++++++++++++---- .../0_stateless/00398_url_functions.reference | 8 ++++++ .../0_stateless/00398_url_functions.sql | 8 ++++++ .../00038_uniq_state_merge2.reference | 4 +-- 4 files changed, 39 insertions(+), 7 deletions(-) diff --git a/src/Functions/URL/domain.h b/src/Functions/URL/domain.h index 1245bb20182..92f58fc91e0 100644 --- a/src/Functions/URL/domain.h +++ b/src/Functions/URL/domain.h @@ -74,20 +74,30 @@ exloop: if ((scheme_end - pos) > 2 && *pos == ':' && *(pos + 1) == '/' && *(pos } Pos dot_pos = nullptr; + Pos colon_pos = nullptr; + bool has_at_symbol = false; + bool has_terminator_after_colon = false; const auto * start_of_host = pos; for (; pos < end; ++pos) { switch (*pos) { case '.': - dot_pos = pos; + if (has_at_symbol || colon_pos == nullptr) + dot_pos = pos; break; - case ':': /// end symbols - case '/': + case ':': + if (has_at_symbol || colon_pos) goto done; + colon_pos = pos; + break; + case '/': /// end symbols case '?': case '#': - return checkAndReturnHost(pos, dot_pos, start_of_host); + goto done; case '@': /// myemail@gmail.com + if (has_terminator_after_colon) return std::string_view{}; + if (has_at_symbol) goto done; + has_at_symbol = true; start_of_host = pos + 1; break; case ' ': /// restricted symbols in whole URL @@ -106,10 +116,16 @@ exloop: if ((scheme_end - pos) > 2 && *pos == ':' && *(pos + 1) == '/' && *(pos case ';': case '=': case '&': - return std::string_view{}; + if (colon_pos == nullptr) + return std::string_view{}; + else + has_terminator_after_colon = true; } } +done: + if (!has_at_symbol) + pos = colon_pos ? colon_pos : pos; return checkAndReturnHost(pos, dot_pos, start_of_host); } diff --git a/tests/queries/0_stateless/00398_url_functions.reference b/tests/queries/0_stateless/00398_url_functions.reference index feba95fb1b3..db3d9224515 100644 --- a/tests/queries/0_stateless/00398_url_functions.reference +++ b/tests/queries/0_stateless/00398_url_functions.reference @@ -7,6 +7,14 @@ http ====HOST==== www.example.com +example.com +example.com +example.com +example.com +example.com +example.com + + www.example.com 127.0.0.1 diff --git a/tests/queries/0_stateless/00398_url_functions.sql b/tests/queries/0_stateless/00398_url_functions.sql index 66fe591bb58..6501b723d23 100644 --- a/tests/queries/0_stateless/00398_url_functions.sql +++ b/tests/queries/0_stateless/00398_url_functions.sql @@ -8,6 +8,14 @@ SELECT protocol('//127.0.0.1:443/') AS Scheme; SELECT '====HOST===='; SELECT domain('http://paul@www.example.com:80/') AS Host; +SELECT domain('user:password@example.com:8080') AS Host; +SELECT domain('http://user:password@example.com:8080') AS Host; +SELECT domain('http://user:password@example.com:8080/path?query=value#fragment') AS Host; +SELECT domain('newuser:@example.com') AS Host; +SELECT domain('http://:pass@example.com') AS Host; +SELECT domain(':newpass@example.com') AS Host; +SELECT domain('http://user:pass@example@.com') AS Host; +SELECT domain('http://user:pass:example.com') AS Host; SELECT domain('http:/paul/example/com') AS Host; SELECT domain('http://www.example.com?q=4') AS Host; SELECT domain('http://127.0.0.1:443/') AS Host; diff --git a/tests/queries/1_stateful/00038_uniq_state_merge2.reference b/tests/queries/1_stateful/00038_uniq_state_merge2.reference index 0b335faaa67..fea0619e225 100644 --- a/tests/queries/1_stateful/00038_uniq_state_merge2.reference +++ b/tests/queries/1_stateful/00038_uniq_state_merge2.reference @@ -1,5 +1,5 @@ -ru 262914 69218 - 91872 89417 +ru 262915 69218 + 91871 89417 com 63298 30285 ua 29037 17475 html 25079 15039 From c0be79f26cb78a753e2e0fb6c8090982262cefaa Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Wed, 19 Oct 2022 12:27:00 +0200 Subject: [PATCH 030/188] CompiledExpressionCache enabled --- programs/local/LocalServer.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e7bc019f597..13573bfe7d5 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -585,6 +586,17 @@ void LocalServer::processConfig() size_t mmap_cache_size = config().getUInt64("mmap_cache_size", 1000); /// The choice of default is arbitrary. if (mmap_cache_size) global_context->setMMappedFileCache(mmap_cache_size); + +#if USE_EMBEDDED_COMPILER + /// 128 MB + constexpr size_t compiled_expression_cache_size_default = 1024 * 1024 * 128; + size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", compiled_expression_cache_size_default); + + constexpr size_t compiled_expression_cache_elements_size_default = 10000; + size_t compiled_expression_cache_elements_size = config().getUInt64("compiled_expression_cache_elements_size", compiled_expression_cache_elements_size_default); + + CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_size, compiled_expression_cache_elements_size); +#endif /// Load global settings from default_profile and system_profile. global_context->setDefaultProfiles(config()); From 0aa26db64c6525f1d4a70e9523277b01133a42ca Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Wed, 19 Oct 2022 12:38:20 +0200 Subject: [PATCH 031/188] Added end of line co comply with Clangformat --- programs/local/LocalServer.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 13573bfe7d5..2ec131b2d59 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -8,10 +8,10 @@ #include #include #include +#include +#include #include #include -#include -#include #include #include #include @@ -593,7 +593,8 @@ void LocalServer::processConfig() size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", compiled_expression_cache_size_default); constexpr size_t compiled_expression_cache_elements_size_default = 10000; - size_t compiled_expression_cache_elements_size = config().getUInt64("compiled_expression_cache_elements_size", compiled_expression_cache_elements_size_default); + size_t compiled_expression_cache_elements_size + = config().getUInt64("compiled_expression_cache_elements_size", compiled_expression_cache_elements_size_default); CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_size, compiled_expression_cache_elements_size); #endif From 632965212fd5a987af80c76b01f83b7dbd359b1f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Oct 2022 12:52:32 +0200 Subject: [PATCH 032/188] Add error "Destination table is myself" to exception list in BC check --- docker/test/stress/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 6b9954c2431..01918fdd21d 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -464,6 +464,7 @@ else -e "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part" \ -e "The set of parts restored in place of" \ -e "(ReplicatedMergeTreeAttachThread): Initialization failed. Error" \ + -e "Code: 269. DB::Exception: Destination table is myself" \ /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv From 2ca0d37e83b9a3c1bee0c4c1cc72fdf3464d4332 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Oct 2022 13:25:03 +0200 Subject: [PATCH 033/188] Get current clickhouse version without sending query in BC check --- docker/test/stress/run.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 6b9954c2431..cf088c5fcd4 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -270,10 +270,6 @@ clickhouse-client --query "SELECT 'Server successfully started', 'OK'" >> /test_ || (echo -e 'Server failed to start (see application_errors.txt and clickhouse-server.clean.log)\tFAIL' >> /test_output/test_results.tsv \ && grep -a ".*Application" /var/log/clickhouse-server/clickhouse-server.log > /test_output/application_errors.txt) -echo "Get previous release tag" -previous_release_tag=$(clickhouse-client --query="SELECT version()" | get_previous_release_tag) -echo $previous_release_tag - stop [ -f /var/log/clickhouse-server/clickhouse-server.log ] || echo -e "Server log does not exist\tFAIL" @@ -331,6 +327,10 @@ zgrep -Fa " received signal " /test_output/gdb.log > /dev/null \ echo -e "Backward compatibility check\n" +echo "Get previous release tag" +previous_release_tag=$(clickhouse-client --version | grep -o "[0-9]*\.[0-9]*\.[0-9]*\.[0-9]*" | get_previous_release_tag) +echo $previous_release_tag + echo "Clone previous release repository" git clone https://github.com/ClickHouse/ClickHouse.git --no-tags --progress --branch=$previous_release_tag --no-recurse-submodules --depth=1 previous_release_repository From 168df7d9ba47a0cf2c4b646760b8427a23b0b8ab Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Wed, 19 Oct 2022 13:34:31 +0200 Subject: [PATCH 034/188] Removed trailing whitespace --- programs/local/LocalServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 2ec131b2d59..531dbc42717 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -586,7 +586,7 @@ void LocalServer::processConfig() size_t mmap_cache_size = config().getUInt64("mmap_cache_size", 1000); /// The choice of default is arbitrary. if (mmap_cache_size) global_context->setMMappedFileCache(mmap_cache_size); - + #if USE_EMBEDDED_COMPILER /// 128 MB constexpr size_t compiled_expression_cache_size_default = 1024 * 1024 * 128; From e5408aac9916495be845b3345d58b86b17c81421 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 19 Oct 2022 12:34:20 +0000 Subject: [PATCH 035/188] Simplify logic --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 + src/Storages/MergeTree/MergeTreeSettings.h | 3 +- .../ReplicatedMergeTreeRestartingThread.cpp | 2 - .../MergeTree/SimpleMergeSelector.cpp | 3 + src/Storages/MergeTree/SimpleMergeSelector.h | 5 ++ src/Storages/StorageMergeTree.cpp | 54 ------------------ src/Storages/StorageMergeTree.h | 11 ---- src/Storages/StorageReplicatedMergeTree.cpp | 56 +------------------ src/Storages/StorageReplicatedMergeTree.h | 14 +---- .../configs/merge_tree.xml | 5 -- .../__init__.py | 0 .../configs/merge_tree.xml | 5 ++ .../configs/zookeeper_config.xml | 0 .../test.py | 12 ++-- 14 files changed, 26 insertions(+), 146 deletions(-) delete mode 100644 tests/integration/test_auto_optimize_partitions/configs/merge_tree.xml rename tests/integration/{test_auto_optimize_partitions => test_merge_tree_optimize_old_parts}/__init__.py (100%) create mode 100644 tests/integration/test_merge_tree_optimize_old_parts/configs/merge_tree.xml rename tests/integration/{test_auto_optimize_partitions => test_merge_tree_optimize_old_parts}/configs/zookeeper_config.xml (100%) rename tests/integration/{test_auto_optimize_partitions => test_merge_tree_optimize_old_parts}/test.py (82%) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 1a5c94a2e26..31fd2919a5d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -333,6 +333,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( SimpleMergeSelector::Settings merge_settings; /// Override value from table settings merge_settings.max_parts_to_merge_at_once = data_settings->max_parts_to_merge_at_once; + merge_settings.min_age_to_force_merge = data_settings->min_age_to_force_merge_seconds; if (aggressive) merge_settings.base = 1; @@ -350,6 +351,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( *out_disable_reason = "There is no need to merge parts according to merge selector algorithm"; return SelectPartsDecision::CANNOT_SELECT; } + } MergeTreeData::DataPartsVector parts; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 63421c3de5d..2f2eda42970 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -62,8 +62,7 @@ struct Settings; M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \ M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ M(UInt64, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds, 1ULL * 3600 * 24 * 30, "Remove old broken detached parts in the background if they remained intouched for a specified by this setting period of time.", 0) \ - M(UInt64, auto_optimize_partition_interval_seconds, 0, "The period of executing the auto optimize partitions in background. Set to 0 to disable.", 0) \ - M(UInt64, auto_optimize_partition_after_seconds, 0, "The number of seconds since last mutation required for partition to be automatically optimized. Set to 0 to disable.", 0) \ + M(UInt64, min_age_to_force_merge_seconds, 0, "If all parts in a certain range are older than this value, range will be always eligable for merging. Set to 0 to disable.", 0) \ M(UInt64, merge_tree_enable_clear_old_broken_detached, false, "Enable clearing old broken detached parts operation in background.", 0) \ M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \ \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 0d021e39a71..e2b23d75746 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -155,8 +155,6 @@ bool ReplicatedMergeTreeRestartingThread::runImpl() storage.mutations_updating_task->activateAndSchedule(); storage.mutations_finalizing_task->activateAndSchedule(); storage.merge_selecting_task->activateAndSchedule(); - if (storage.auto_optimize_partition_task) - storage.auto_optimize_partition_task->activateAndSchedule(); storage.cleanup_thread.start(); storage.part_check_thread.start(); diff --git a/src/Storages/MergeTree/SimpleMergeSelector.cpp b/src/Storages/MergeTree/SimpleMergeSelector.cpp index 3b71e2720c8..f9ed6aedc60 100644 --- a/src/Storages/MergeTree/SimpleMergeSelector.cpp +++ b/src/Storages/MergeTree/SimpleMergeSelector.cpp @@ -102,6 +102,9 @@ bool allow( double max_size_to_lower_base_log, const SimpleMergeSelector::Settings & settings) { + if (settings.min_age_to_force_merge && min_age >= settings.min_age_to_force_merge) + return true; + // std::cerr << "sum_size: " << sum_size << "\n"; /// Map size to 0..1 using logarithmic scale diff --git a/src/Storages/MergeTree/SimpleMergeSelector.h b/src/Storages/MergeTree/SimpleMergeSelector.h index 11ffe8b672a..1480afaf1d2 100644 --- a/src/Storages/MergeTree/SimpleMergeSelector.h +++ b/src/Storages/MergeTree/SimpleMergeSelector.h @@ -141,6 +141,11 @@ public: double heuristic_to_align_parts_max_absolute_difference_in_powers_of_two = 0.5; double heuristic_to_align_parts_max_score_adjustment = 0.75; + /// If it's not 0, all part ranges that have min_age larger than min_age_to_force_merge + /// will be considered for merging + size_t min_age_to_force_merge = 0; + + /** Heuristic: * From right side of range, remove all parts, that size is less than specified ratio of sum_size. */ diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 1af0e6e756f..a65af1cf69e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -798,54 +798,6 @@ void StorageMergeTree::loadMutations() increment.value = std::max(increment.value.load(), current_mutations_by_version.rbegin()->first); } -std::shared_ptr StorageMergeTree::selectOnePartitionToOptimize( - const StorageMetadataPtr & metadata_snapshot, - TableLockHolder & table_lock_holder, - std::unique_lock & lock, - const MergeTreeTransactionPtr & txn, - bool optimize_skip_merged_partitions) -{ - // Select the `best partition to merge. - std::unordered_map partition_parts_sum_diff; - ssize_t base = time(nullptr) - getSettings()->auto_optimize_partition_after_seconds; - auto data_parts = getDataPartsForInternalUsage(); - for (const auto & part : data_parts) - { - if (part->modification_time < base) - partition_parts_sum_diff[part->info.partition_id] += (base - part->modification_time); - } - auto best_partition_it = std::max_element(partition_parts_sum_diff.begin(), partition_parts_sum_diff.end(), [](const auto & e1, const auto & e2) { return e1.second < e2.second; }); - if (best_partition_it == partition_parts_sum_diff.end()) - { - return nullptr; - } - // Merge the selected partition. - String disable_reason; - SelectPartsDecision select_decision; - auto merge_entry = selectPartsToMerge( - metadata_snapshot, - true, - best_partition_it->first, - true, - &disable_reason, - table_lock_holder, - lock, - txn, - optimize_skip_merged_partitions, - &select_decision); - - if (select_decision == SelectPartsDecision::NOTHING_TO_MERGE) - return nullptr; - if (!merge_entry) - { - static constexpr const char * message = "Cannot OPTIMIZE table in background: {}"; - if (disable_reason.empty()) - disable_reason = "unknown reason"; - LOG_INFO(log, message, disable_reason); - } - return merge_entry; -} - MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( const StorageMetadataPtr & metadata_snapshot, bool aggressive, @@ -1203,12 +1155,6 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign if (!merge_entry) mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr, share_lock, lock); - if (getSettings()->auto_optimize_partition_interval_seconds - && !merge_entry && !mutate_entry - && time_after_previous_optimize_one_partition.compareAndRestartDeferred(getSettings()->auto_optimize_partition_interval_seconds)) - { - merge_entry = selectOnePartitionToOptimize(metadata_snapshot, share_lock, lock, txn); - } has_mutations = !current_mutations_by_version.empty(); } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 5d7a24a2f7e..ea2527e44a7 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -135,8 +135,6 @@ private: AtomicStopwatch time_after_previous_cleanup_temporary_directories; /// For clearOldBrokenDetachedParts AtomicStopwatch time_after_previous_cleanup_broken_detached_parts; - /// For optimizeOnePartition; - AtomicStopwatch time_after_previous_optimize_one_partition; /// Mutex for parts currently processing in background /// merging (also with TTL), mutating or moving. @@ -171,15 +169,6 @@ private: String * out_disable_reason = nullptr, bool optimize_skip_merged_partitions = false); - /** Determines what part within one partition should be merged. - */ - std::shared_ptr selectOnePartitionToOptimize( - const StorageMetadataPtr & metadata_snapshot, - TableLockHolder & table_lock_holder, - std::unique_lock & lock, - const MergeTreeTransactionPtr & txn, - bool optimize_skip_merged_partitions = false); - /// Make part state outdated and queue it to remove without timeout /// If force, then stop merges and block them until part state became outdated. Throw exception if part doesn't exists /// If not force, then take merges selector and check that part is not participating in background operations. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 186843d3bd6..5818ff37c5b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -293,14 +293,9 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( merge_selecting_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mergeSelectingTask)", [this] { mergeSelectingTask(); }); - if (getSettings()->auto_optimize_partition_interval_seconds) - auto_optimize_partition_task = getContext()->getSchedulePool().createTask( - getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::autoOptimizePartitionTask)", [this] { autoOptimizePartitionTask(); }); /// Will be activated if we win leader election. merge_selecting_task->deactivate(); - if (auto_optimize_partition_task) - auto_optimize_partition_task->deactivate(); mutations_finalizing_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mutationsFinalizingTask)", [this] { mutationsFinalizingTask(); }); @@ -4457,31 +4452,9 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con local_context); } -void StorageReplicatedMergeTree::autoOptimizePartitionTask() -{ - if (!is_leader) - return; - - auto table_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); - try - { - optimizeImpl(nullptr, nullptr, nullptr, true, true, Names{}, table_lock, getContext(), true); - } - catch (const Exception & e) - { - LOG_ERROR(log, "Can't optimize partitions automatically for table: {}, reason: {}", getStorageID().getNameForLogs(), e.displayText()); - } - catch (...) - { - LOG_ERROR(log, "There is a problem when optimizing table: {}, reason: {}", getStorageID().getNameForLogs(), getCurrentExceptionMessage(true)); - } - if (auto_optimize_partition_task) - auto_optimize_partition_task->scheduleAfter(getSettings()->auto_optimize_partition_interval_seconds); -} - bool StorageReplicatedMergeTree::optimize( - const ASTPtr & query, - const StorageMetadataPtr & metadata_snapshot, + const ASTPtr &, + const StorageMetadataPtr &, const ASTPtr & partition, bool final, bool deduplicate, @@ -4491,20 +4464,7 @@ bool StorageReplicatedMergeTree::optimize( /// NOTE: exclusive lock cannot be used here, since this may lead to deadlock (see comments below), /// but it should be safe to use non-exclusive to avoid dropping parts that may be required for processing queue. auto table_lock = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef().lock_acquire_timeout); - return optimizeImpl(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, table_lock, query_context, false); -} -bool StorageReplicatedMergeTree::optimizeImpl( - const ASTPtr &, - const StorageMetadataPtr &, - const ASTPtr & partition, - bool final, - bool deduplicate, - const Names & deduplicate_by_columns, - TableLockHolder & table_lock, - ContextPtr query_context, - bool auto_optimize_in_background) -{ assertNotReadonly(); if (!is_leader) @@ -4602,12 +4562,8 @@ bool StorageReplicatedMergeTree::optimizeImpl( DataPartsVector data_parts = getVisibleDataPartsVector(query_context); std::unordered_set partition_ids; - ssize_t baseline = time(nullptr) - storage_settings_ptr->auto_optimize_partition_after_seconds; for (const DataPartPtr & part : data_parts) - { - if (!auto_optimize_in_background || part->modification_time < baseline) - partition_ids.emplace(part->info.partition_id); - } + partition_ids.emplace(part->info.partition_id); for (const String & partition_id : partition_ids) { @@ -4626,12 +4582,6 @@ bool StorageReplicatedMergeTree::optimizeImpl( table_lock.reset(); - if (!auto_optimize_in_background) - { - for (auto & merge_entry : merge_entries) - waitForLogEntryToBeProcessedIfNecessary(merge_entry, query_context); - } - return assigned; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 1d315f42ad0..b88b9497e39 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -331,16 +331,7 @@ public: bool canUseZeroCopyReplication() const; private: std::atomic_bool are_restoring_replica {false}; - bool optimizeImpl( - const ASTPtr &, - const StorageMetadataPtr &, - const ASTPtr & partition, - bool final, - bool deduplicate, - const Names & deduplicate_by_columns, - TableLockHolder & table_lock, - ContextPtr query_context, - bool auto_optimize_in_background); + /// Get a sequential consistent view of current parts. ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const; @@ -455,9 +446,6 @@ private: /// A task that marks finished mutations as done. BackgroundSchedulePool::TaskHolder mutations_finalizing_task; - /// A task that optimize partitions automatically background if enabled. - BackgroundSchedulePool::TaskHolder auto_optimize_partition_task { nullptr }; - /// A thread that removes old parts, log entries, and blocks. ReplicatedMergeTreeCleanupThread cleanup_thread; diff --git a/tests/integration/test_auto_optimize_partitions/configs/merge_tree.xml b/tests/integration/test_auto_optimize_partitions/configs/merge_tree.xml deleted file mode 100644 index 9c3da3c99ac..00000000000 --- a/tests/integration/test_auto_optimize_partitions/configs/merge_tree.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - 2 - - diff --git a/tests/integration/test_auto_optimize_partitions/__init__.py b/tests/integration/test_merge_tree_optimize_old_parts/__init__.py similarity index 100% rename from tests/integration/test_auto_optimize_partitions/__init__.py rename to tests/integration/test_merge_tree_optimize_old_parts/__init__.py diff --git a/tests/integration/test_merge_tree_optimize_old_parts/configs/merge_tree.xml b/tests/integration/test_merge_tree_optimize_old_parts/configs/merge_tree.xml new file mode 100644 index 00000000000..07c96a8e41a --- /dev/null +++ b/tests/integration/test_merge_tree_optimize_old_parts/configs/merge_tree.xml @@ -0,0 +1,5 @@ + + + 2 + + diff --git a/tests/integration/test_auto_optimize_partitions/configs/zookeeper_config.xml b/tests/integration/test_merge_tree_optimize_old_parts/configs/zookeeper_config.xml similarity index 100% rename from tests/integration/test_auto_optimize_partitions/configs/zookeeper_config.xml rename to tests/integration/test_merge_tree_optimize_old_parts/configs/zookeeper_config.xml diff --git a/tests/integration/test_auto_optimize_partitions/test.py b/tests/integration/test_merge_tree_optimize_old_parts/test.py similarity index 82% rename from tests/integration/test_auto_optimize_partitions/test.py rename to tests/integration/test_merge_tree_optimize_old_parts/test.py index 77ce56e3787..e78ecbeed60 100644 --- a/tests/integration/test_auto_optimize_partitions/test.py +++ b/tests/integration/test_merge_tree_optimize_old_parts/test.py @@ -38,21 +38,21 @@ def check_expected_result_or_fail(seconds, expected): assert ok -def test_without_auto_optimize_merge_tree(start_cluster): +def test_without_force_merge_old_parts(start_cluster): node.query("CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i;") node.query("INSERT INTO test SELECT 1") node.query("INSERT INTO test SELECT 2") node.query("INSERT INTO test SELECT 3") expected = TSV("""3\n""") - check_expected_result_or_fail(5, expected) + check_expected_result_or_fail(10, expected) node.query("DROP TABLE test;") -def test_auto_optimize_merge_tree(start_cluster): +def test_force_merge_old_parts(start_cluster): node.query( - "CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i SETTINGS auto_optimize_partition_interval_seconds=5;" + "CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i SETTINGS min_age_to_force_merge_seconds=5;" ) node.query("INSERT INTO test SELECT 1") node.query("INSERT INTO test SELECT 2") @@ -64,9 +64,9 @@ def test_auto_optimize_merge_tree(start_cluster): node.query("DROP TABLE test;") -def test_auto_optimize_replicated_merge_tree(start_cluster): +def test_force_merge_old_parts_replicated_merge_tree(start_cluster): node.query( - "CREATE TABLE test (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') ORDER BY i SETTINGS auto_optimize_partition_interval_seconds=5;" + "CREATE TABLE test (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') ORDER BY i SETTINGS min_age_to_force_merge_seconds=5;" ) node.query("INSERT INTO test SELECT 1") node.query("INSERT INTO test SELECT 2") From 05e6b94b5411b14955814ad55cb9ed0b3277b302 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 19 Oct 2022 12:57:51 +0000 Subject: [PATCH 036/188] Polishing --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 1 - src/Storages/MergeTree/MergeTreeSettings.h | 2 +- src/Storages/MergeTree/SimpleMergeSelector.h | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++ src/Storages/StorageReplicatedMergeTree.h | 3 - .../configs/merge_tree.xml | 5 -- .../test.py | 62 +++++++++++-------- 7 files changed, 44 insertions(+), 39 deletions(-) delete mode 100644 tests/integration/test_merge_tree_optimize_old_parts/configs/merge_tree.xml diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 31fd2919a5d..b0ef1522685 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -351,7 +351,6 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( *out_disable_reason = "There is no need to merge parts according to merge selector algorithm"; return SelectPartsDecision::CANNOT_SELECT; } - } MergeTreeData::DataPartsVector parts; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 2f2eda42970..3fecb85f484 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -62,7 +62,7 @@ struct Settings; M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \ M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ M(UInt64, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds, 1ULL * 3600 * 24 * 30, "Remove old broken detached parts in the background if they remained intouched for a specified by this setting period of time.", 0) \ - M(UInt64, min_age_to_force_merge_seconds, 0, "If all parts in a certain range are older than this value, range will be always eligable for merging. Set to 0 to disable.", 0) \ + M(UInt64, min_age_to_force_merge_seconds, 0, "If all parts in a certain range are older than this value, range will be always eligible for merging. Set to 0 to disable.", 0) \ M(UInt64, merge_tree_enable_clear_old_broken_detached, false, "Enable clearing old broken detached parts operation in background.", 0) \ M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \ \ diff --git a/src/Storages/MergeTree/SimpleMergeSelector.h b/src/Storages/MergeTree/SimpleMergeSelector.h index 1480afaf1d2..c20eaa6e8de 100644 --- a/src/Storages/MergeTree/SimpleMergeSelector.h +++ b/src/Storages/MergeTree/SimpleMergeSelector.h @@ -141,11 +141,11 @@ public: double heuristic_to_align_parts_max_absolute_difference_in_powers_of_two = 0.5; double heuristic_to_align_parts_max_score_adjustment = 0.75; - /// If it's not 0, all part ranges that have min_age larger than min_age_to_force_merge - /// will be considered for merging + /** If it's not 0, all part ranges that have min_age larger than min_age_to_force_merge + * will be considered for merging + */ size_t min_age_to_force_merge = 0; - /** Heuristic: * From right side of range, remove all parts, that size is less than specified ratio of sum_size. */ diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5818ff37c5b..7a2ff56a782 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4452,6 +4452,7 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con local_context); } + bool StorageReplicatedMergeTree::optimize( const ASTPtr &, const StorageMetadataPtr &, @@ -4582,6 +4583,9 @@ bool StorageReplicatedMergeTree::optimize( table_lock.reset(); + for (auto & merge_entry : merge_entries) + waitForLogEntryToBeProcessedIfNecessary(merge_entry, query_context); + return assigned; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index b88b9497e39..e10ffcce22c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -587,9 +587,6 @@ private: void mutationsUpdatingTask(); - /** Auto optimize partitions - */ - void autoOptimizePartitionTask(); /** Clone data from another replica. * If replica can not be cloned throw Exception. */ diff --git a/tests/integration/test_merge_tree_optimize_old_parts/configs/merge_tree.xml b/tests/integration/test_merge_tree_optimize_old_parts/configs/merge_tree.xml deleted file mode 100644 index 07c96a8e41a..00000000000 --- a/tests/integration/test_merge_tree_optimize_old_parts/configs/merge_tree.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - 2 - - diff --git a/tests/integration/test_merge_tree_optimize_old_parts/test.py b/tests/integration/test_merge_tree_optimize_old_parts/test.py index e78ecbeed60..7b386eba2c4 100644 --- a/tests/integration/test_merge_tree_optimize_old_parts/test.py +++ b/tests/integration/test_merge_tree_optimize_old_parts/test.py @@ -7,7 +7,7 @@ from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", - main_configs=["configs/zookeeper_config.xml", "configs/merge_tree.xml"], + main_configs=["configs/zookeeper_config.xml"], with_zookeeper=True, ) @@ -22,57 +22,67 @@ def start_cluster(): cluster.shutdown() -def check_expected_result_or_fail(seconds, expected): +def get_part_number(table_name): + return TSV( + node.query( + f"SELECT count(*) FROM system.parts where table='{table_name}' and active=1" + ) + ) + + +def check_expected_part_number(seconds, table_name, expected): ok = False for i in range(int(seconds) * 2): - result = TSV( - node.query( - "SELECT count(*) FROM system.parts where table='test' and active=1" - ) - ) + result = get_part_number(table_name) if result == expected: ok = True break else: - time.sleep(0.5) + time.sleep(1) assert ok def test_without_force_merge_old_parts(start_cluster): - node.query("CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i;") - node.query("INSERT INTO test SELECT 1") - node.query("INSERT INTO test SELECT 2") - node.query("INSERT INTO test SELECT 3") + node.query( + "CREATE TABLE test_without_merge (i Int64) ENGINE = MergeTree ORDER BY i;" + ) + node.query("INSERT INTO test_without_merge SELECT 1") + node.query("INSERT INTO test_without_merge SELECT 2") + node.query("INSERT INTO test_without_merge SELECT 3") expected = TSV("""3\n""") - check_expected_result_or_fail(10, expected) + # verify that the parts don't get merged + for i in range(10): + if get_part_number("test_without_merge") != expected: + assert False + time.sleep(1) - node.query("DROP TABLE test;") + node.query("DROP TABLE test_without_merge;") def test_force_merge_old_parts(start_cluster): node.query( - "CREATE TABLE test (i Int64) ENGINE = MergeTree ORDER BY i SETTINGS min_age_to_force_merge_seconds=5;" + "CREATE TABLE test_with_merge (i Int64) ENGINE = MergeTree ORDER BY i SETTINGS min_age_to_force_merge_seconds=5;" ) - node.query("INSERT INTO test SELECT 1") - node.query("INSERT INTO test SELECT 2") - node.query("INSERT INTO test SELECT 3") + node.query("INSERT INTO test_with_merge SELECT 1") + node.query("INSERT INTO test_with_merge SELECT 2") + node.query("INSERT INTO test_with_merge SELECT 3") expected = TSV("""1\n""") - check_expected_result_or_fail(10, expected) + check_expected_part_number(10, "test_with_merge", expected) - node.query("DROP TABLE test;") + node.query("DROP TABLE test_with_merge;") def test_force_merge_old_parts_replicated_merge_tree(start_cluster): node.query( - "CREATE TABLE test (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') ORDER BY i SETTINGS min_age_to_force_merge_seconds=5;" + "CREATE TABLE test_replicated (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/testing/test', 'node') ORDER BY i SETTINGS min_age_to_force_merge_seconds=5;" ) - node.query("INSERT INTO test SELECT 1") - node.query("INSERT INTO test SELECT 2") - node.query("INSERT INTO test SELECT 3") + node.query("INSERT INTO test_replicated SELECT 1") + node.query("INSERT INTO test_replicated SELECT 2") + node.query("INSERT INTO test_replicated SELECT 3") expected = TSV("""1\n""") - check_expected_result_or_fail(10, expected) + check_expected_part_number(10, "test_replicated", expected) - node.query("DROP TABLE test;") + node.query("DROP TABLE test_replicated;") From 5b3b11b517d79552c3956dc267bb16930113639d Mon Sep 17 00:00:00 2001 From: Constantine Peresypkin Date: Sun, 25 Sep 2022 19:58:00 +0200 Subject: [PATCH 037/188] add Morton Coding (ZCurve) --- .gitmodules | 3 + cmake/cpu_features.cmake | 16 + contrib/CMakeLists.txt | 1 + contrib/morton-nd | 1 + contrib/morton-nd-cmake/CMakeLists.txt | 3 + docker/test/fasttest/run.sh | 1 + src/Functions/CMakeLists.txt | 1 + src/Functions/mortonDecode.cpp | 433 ++++++++++++++++++ src/Functions/mortonEncode.cpp | 393 ++++++++++++++++ .../0_stateless/02457_morton_coding.reference | 12 + .../0_stateless/02457_morton_coding.sql | 137 ++++++ .../02457_morton_coding_with_mask.reference | 15 + .../02457_morton_coding_with_mask.sql | 143 ++++++ 13 files changed, 1159 insertions(+) create mode 160000 contrib/morton-nd create mode 100644 contrib/morton-nd-cmake/CMakeLists.txt create mode 100644 src/Functions/mortonDecode.cpp create mode 100644 src/Functions/mortonEncode.cpp create mode 100644 tests/queries/0_stateless/02457_morton_coding.reference create mode 100644 tests/queries/0_stateless/02457_morton_coding.sql create mode 100644 tests/queries/0_stateless/02457_morton_coding_with_mask.reference create mode 100644 tests/queries/0_stateless/02457_morton_coding_with_mask.sql diff --git a/.gitmodules b/.gitmodules index abd29c38846..293029ad171 100644 --- a/.gitmodules +++ b/.gitmodules @@ -287,3 +287,6 @@ [submodule "contrib/corrosion"] path = contrib/corrosion url = https://github.com/corrosion-rs/corrosion.git +[submodule "contrib/morton-nd"] + path = contrib/morton-nd + url = https://github.com/morton-nd/morton-nd diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 6707d703372..aef31251a56 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -75,6 +75,7 @@ elseif (ARCH_AMD64) option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0) option (ENABLE_AVX512_VBMI "Use AVX512_VBMI instruction on x86_64 (depends on ENABLE_AVX512)" 0) option (ENABLE_BMI "Use BMI instructions on x86_64" 0) + option (ENABLE_BMI2 "Use BMI2 instructions on x86_64 (depends on ENABLE_AVX2)" 0) option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 0) option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) @@ -90,6 +91,7 @@ elseif (ARCH_AMD64) SET(ENABLE_AVX512 0) SET(ENABLE_AVX512_VBMI 0) SET(ENABLE_BMI 0) + SET(ENABLE_BMI2 0) SET(ENABLE_AVX2_FOR_SPEC_OP 0) SET(ENABLE_AVX512_FOR_SPEC_OP 0) endif() @@ -237,6 +239,20 @@ elseif (ARCH_AMD64) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () + set (TEST_FLAG "-mbmi2") + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + #include + int main() { + auto a = _pdep_u64(0, 0); + (void)a; + return 0; + } + " HAVE_BMI2) + if (HAVE_BMI2 AND HAVE_AVX2 AND ENABLE_AVX2 AND ENABLE_BMI2) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () + # Limit avx2/avx512 flag for specific source build set (X86_INTRINSICS_FLAGS "") if (ENABLE_AVX2_FOR_SPEC_OP) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index f914c0d2d3f..3017c1a8fc0 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -164,6 +164,7 @@ add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) add_contrib (c-ares-cmake c-ares) add_contrib (qpl-cmake qpl) +add_contrib (morton-nd-cmake morton-nd) add_contrib(annoy-cmake annoy) diff --git a/contrib/morton-nd b/contrib/morton-nd new file mode 160000 index 00000000000..3795491a4aa --- /dev/null +++ b/contrib/morton-nd @@ -0,0 +1 @@ +Subproject commit 3795491a4aa3cdc916c8583094683f0d68df5bc0 diff --git a/contrib/morton-nd-cmake/CMakeLists.txt b/contrib/morton-nd-cmake/CMakeLists.txt new file mode 100644 index 00000000000..4842781503f --- /dev/null +++ b/contrib/morton-nd-cmake/CMakeLists.txt @@ -0,0 +1,3 @@ +add_library(_morton_nd INTERFACE) +target_include_directories(_morton_nd SYSTEM BEFORE INTERFACE "${ClickHouse_SOURCE_DIR}/contrib/morton-nd/include/") +add_library(ch_contrib::morton_nd ALIAS _morton_nd) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 9d6cf22c817..de9125d565b 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -136,6 +136,7 @@ function clone_submodules contrib/wyhash contrib/hashidsxx contrib/c-ares + contrib/morton-nd ) git submodule sync diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index ad36c51447f..c84e23da85b 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -22,6 +22,7 @@ list (APPEND PUBLIC_LIBS ch_contrib::metrohash ch_contrib::murmurhash ch_contrib::hashidsxx + ch_contrib::morton_nd ) list (APPEND PRIVATE_LIBS diff --git a/src/Functions/mortonDecode.cpp b/src/Functions/mortonDecode.cpp new file mode 100644 index 00000000000..337fd5e3a38 --- /dev/null +++ b/src/Functions/mortonDecode.cpp @@ -0,0 +1,433 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#if USE_MULTITARGET_CODE && defined(__BMI2__) +#include +#endif + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int ARGUMENT_OUT_OF_BOUND; +} + +#define EXTRACT_VECTOR(INDEX) \ + auto col##INDEX = ColumnUInt64::create(); \ + auto & vec##INDEX = col##INDEX->getData(); \ + vec##INDEX.resize(input_rows_count); + +#define DECODE(ND, ...) \ + if (nd == (ND)) \ + { \ + for (size_t i = 0; i < input_rows_count; i++) \ + { \ + auto res = MortonND_##ND##D_Dec.Decode(col_code->getUInt(i)); \ + __VA_ARGS__ \ + } \ + } + +#define MASK(IDX, ...) \ + ((mask) ? shrink(mask->getColumn((IDX)).getUInt(0), std::get(__VA_ARGS__)) : std::get(__VA_ARGS__)) + +#define EXECUTE() \ + size_t nd; \ + const auto * col_const = typeid_cast(arguments[0].column.get()); \ + const auto * mask = typeid_cast(col_const->getDataColumnPtr().get()); \ + if (mask) \ + nd = mask->tupleSize(); \ + else \ + nd = col_const->getUInt(0); \ + auto non_const_arguments = arguments; \ + non_const_arguments[1].column = non_const_arguments[1].column->convertToFullColumnIfConst(); \ + const ColumnPtr & col_code = non_const_arguments[1].column; \ + Columns tuple_columns(nd); \ + EXTRACT_VECTOR(0) \ + if (nd == 1) \ + { \ + if (mask) \ + { \ + for (size_t i = 0; i < input_rows_count; i++) \ + { \ + vec0[i] = shrink(mask->getColumn(0).getUInt(0), col_code->getUInt(i)); \ + } \ + tuple_columns[0] = std::move(col0); \ + } \ + else \ + { \ + for (size_t i = 0; i < input_rows_count; i++) \ + { \ + vec0[i] = col_code->getUInt(i); \ + } \ + tuple_columns[0] = std::move(col0); \ + } \ + return ColumnTuple::create(tuple_columns); \ + } \ + EXTRACT_VECTOR(1) \ + DECODE(2, \ + vec0[i] = MASK(0, res); \ + vec1[i] = MASK(1, res);) \ + EXTRACT_VECTOR(2) \ + DECODE(3, \ + vec0[i] = MASK(0, res); \ + vec1[i] = MASK(1, res); \ + vec2[i] = MASK(2, res);) \ + EXTRACT_VECTOR(3) \ + DECODE(4, \ + vec0[i] = MASK(0, res); \ + vec1[i] = MASK(1, res); \ + vec2[i] = MASK(2, res); \ + vec3[i] = MASK(3, res);) \ + EXTRACT_VECTOR(4) \ + DECODE(5, \ + vec0[i] = MASK(0, res); \ + vec1[i] = MASK(1, res); \ + vec2[i] = MASK(2, res); \ + vec3[i] = MASK(3, res); \ + vec4[i] = MASK(4, res);) \ + EXTRACT_VECTOR(5) \ + DECODE(6, \ + vec0[i] = MASK(0, res); \ + vec1[i] = MASK(1, res); \ + vec2[i] = MASK(2, res); \ + vec3[i] = MASK(3, res); \ + vec4[i] = MASK(4, res); \ + vec5[i] = MASK(5, res);) \ + EXTRACT_VECTOR(6) \ + DECODE(7, \ + vec0[i] = MASK(0, res); \ + vec1[i] = MASK(1, res); \ + vec2[i] = MASK(2, res); \ + vec3[i] = MASK(3, res); \ + vec4[i] = MASK(4, res); \ + vec5[i] = MASK(5, res); \ + vec6[i] = MASK(6, res);) \ + EXTRACT_VECTOR(7) \ + DECODE(8, \ + vec0[i] = MASK(0, res); \ + vec1[i] = MASK(1, res); \ + vec2[i] = MASK(2, res); \ + vec3[i] = MASK(3, res); \ + vec4[i] = MASK(4, res); \ + vec5[i] = MASK(5, res); \ + vec6[i] = MASK(6, res); \ + vec7[i] = MASK(7, res);) \ + switch (nd) \ + { \ + case 2: \ + tuple_columns[0] = std::move(col0); \ + tuple_columns[1] = std::move(col1); \ + break; \ + case 3: \ + tuple_columns[0] = std::move(col0); \ + tuple_columns[1] = std::move(col1); \ + tuple_columns[2] = std::move(col2); \ + return ColumnTuple::create(tuple_columns); \ + case 4: \ + tuple_columns[0] = std::move(col0); \ + tuple_columns[1] = std::move(col1); \ + tuple_columns[2] = std::move(col2); \ + tuple_columns[3] = std::move(col3); \ + return ColumnTuple::create(tuple_columns); \ + case 5: \ + tuple_columns[0] = std::move(col0); \ + tuple_columns[1] = std::move(col1); \ + tuple_columns[2] = std::move(col2); \ + tuple_columns[3] = std::move(col3); \ + tuple_columns[4] = std::move(col4); \ + return ColumnTuple::create(tuple_columns); \ + case 6: \ + tuple_columns[0] = std::move(col0); \ + tuple_columns[1] = std::move(col1); \ + tuple_columns[2] = std::move(col2); \ + tuple_columns[3] = std::move(col3); \ + tuple_columns[4] = std::move(col4); \ + tuple_columns[5] = std::move(col5); \ + return ColumnTuple::create(tuple_columns); \ + case 7: \ + tuple_columns[0] = std::move(col0); \ + tuple_columns[1] = std::move(col1); \ + tuple_columns[2] = std::move(col2); \ + tuple_columns[3] = std::move(col3); \ + tuple_columns[4] = std::move(col4); \ + tuple_columns[5] = std::move(col5); \ + tuple_columns[6] = std::move(col6); \ + return ColumnTuple::create(tuple_columns); \ + case 8: \ + tuple_columns[0] = std::move(col0); \ + tuple_columns[1] = std::move(col1); \ + tuple_columns[2] = std::move(col2); \ + tuple_columns[3] = std::move(col3); \ + tuple_columns[4] = std::move(col4); \ + tuple_columns[5] = std::move(col5); \ + tuple_columns[6] = std::move(col6); \ + tuple_columns[7] = std::move(col7); \ + return ColumnTuple::create(tuple_columns); \ + } \ + return ColumnTuple::create(tuple_columns); + +DECLARE_DEFAULT_CODE( +constexpr auto MortonND_2D_Dec = mortonnd::MortonNDLutDecoder<2, 32, 8>(); +constexpr auto MortonND_3D_Dec = mortonnd::MortonNDLutDecoder<3, 21, 8>(); +constexpr auto MortonND_4D_Dec = mortonnd::MortonNDLutDecoder<4, 16, 8>(); +constexpr auto MortonND_5D_Dec = mortonnd::MortonNDLutDecoder<5, 12, 8>(); +constexpr auto MortonND_6D_Dec = mortonnd::MortonNDLutDecoder<6, 10, 8>(); +constexpr auto MortonND_7D_Dec = mortonnd::MortonNDLutDecoder<7, 9, 8>(); +constexpr auto MortonND_8D_Dec = mortonnd::MortonNDLutDecoder<8, 8, 8>(); +class FunctionMortonDecode : public IFunction +{ +public: + static constexpr auto name = "mortonDecode"; + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 2; + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + UInt64 tuple_size = 0; + const auto * col_const = typeid_cast(arguments[0].column.get()); + if (!col_const) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column type {} of function {}, should be a constant (UInt or Tuple)", + arguments[0].type->getName(), getName()); + if (!WhichDataType(arguments[1].type).isNativeUInt()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column type {} of function {}, should be a native UInt", + arguments[1].type->getName(), getName()); + const auto * mask = typeid_cast(col_const->getDataColumnPtr().get()); + if (mask) + { + tuple_size = mask->tupleSize(); + } + else if (WhichDataType(arguments[0].type).isNativeUInt()) + { + tuple_size = col_const->getUInt(0); + } + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column type {} of function {}, should be UInt or Tuple", + arguments[0].type->getName(), getName()); + if (tuple_size > 8 || tuple_size < 1) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Illegal first argument for function {}, should be a number in range 1-8 or a Tuple of such size", + getName()); + if (mask) + { + const auto * type_tuple = typeid_cast(arguments[0].type.get()); + for (size_t i = 0; i < tuple_size; i++) + { + if (!WhichDataType(type_tuple->getElement(i)).isNativeUInt()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument in tuple for function {}, should be a native UInt", + type_tuple->getElement(i)->getName(), getName()); + auto ratio = mask->getColumn(i).getUInt(0); + if (ratio > 8 || ratio < 1) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Illegal argument {} in tuple for function {}, should be a number in range 1-8", + ratio, getName()); + } + } + DataTypes types(tuple_size); + for (size_t i = 0; i < tuple_size; i++) + { + types[i] = std::make_shared(); + } + return std::make_shared(types); + } + + static UInt64 shrink(UInt64 ratio, UInt64 value) + { + switch (ratio) + { + case 1: + return value; + case 2: + return std::get<1>(MortonND_2D_Dec.Decode(value)); + case 3: + return std::get<2>(MortonND_3D_Dec.Decode(value)); + case 4: + return std::get<3>(MortonND_4D_Dec.Decode(value)); + case 5: + return std::get<4>(MortonND_5D_Dec.Decode(value)); + case 6: + return std::get<5>(MortonND_6D_Dec.Decode(value)); + case 7: + return std::get<6>(MortonND_7D_Dec.Decode(value)); + case 8: + return std::get<7>(MortonND_8D_Dec.Decode(value)); + } + return value; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + EXECUTE() + } +}; +) // DECLARE_DEFAULT_CODE + +#if defined(MORTON_ND_BMI2_ENABLED) +#undef DECODE +#define DECODE(ND, ...) \ + if (nd == (ND)) \ + { \ + for (size_t i = 0; i < input_rows_count; i++) \ + { \ + auto res = MortonND_##ND##D::Decode(col_code->getUInt(i)); \ + __VA_ARGS__ \ + } \ + } + +DECLARE_AVX2_SPECIFIC_CODE( +using MortonND_2D = mortonnd::MortonNDBmi<2, uint64_t>; +using MortonND_3D = mortonnd::MortonNDBmi<3, uint64_t>; +using MortonND_4D = mortonnd::MortonNDBmi<4, uint64_t>; +using MortonND_5D = mortonnd::MortonNDBmi<5, uint64_t>; +using MortonND_6D = mortonnd::MortonNDBmi<6, uint64_t>; +using MortonND_7D = mortonnd::MortonNDBmi<7, uint64_t>; +using MortonND_8D = mortonnd::MortonNDBmi<8, uint64_t>; +class FunctionMortonDecode: public TargetSpecific::Default::FunctionMortonDecode +{ + static UInt64 shrink(UInt64 ratio, UInt64 value) + { + switch (ratio) + { + case 1: + return value; + case 2: + return std::get<1>(MortonND_2D::Decode(value)); + case 3: + return std::get<2>(MortonND_3D::Decode(value)); + case 4: + return std::get<3>(MortonND_4D::Decode(value)); + case 5: + return std::get<4>(MortonND_5D::Decode(value)); + case 6: + return std::get<5>(MortonND_6D::Decode(value)); + case 7: + return std::get<6>(MortonND_7D::Decode(value)); + case 8: + return std::get<7>(MortonND_8D::Decode(value)); + } + return value; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + EXECUTE() + } +}; +) +#endif // MORTON_ND_BMI2_ENABLED + +#undef DECODE +#undef MASK +#undef EXTRACT_VECTOR +#undef EXECUTE + +class FunctionMortonDecode: public TargetSpecific::Default::FunctionMortonDecode +{ +public: + explicit FunctionMortonDecode(ContextPtr context) : selector(context) + { + selector.registerImplementation(); + +#if USE_MULTITARGET_CODE && defined(MORTON_ND_BMI2_ENABLED) + selector.registerImplementation(); +#endif + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + return selector.selectAndExecute(arguments, result_type, input_rows_count); + } + + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context); + } + +private: + ImplementationSelector selector; +}; + +REGISTER_FUNCTION(MortonDecode) +{ + factory.registerFunction({ + R"( +Decodes a Morton encoding (ZCurve) into the corresponding unsigned integer tuple + +The function has two modes of operation: +- Simple +- Expanded + +Simple: accepts a resulting tuple size as a first argument and the code as a second argument. +[example:simple] +Will decode into: `(1,2,3,4)` +The resulting tuple size cannot be more than 8 + +Expanded: accepts a range mask (tuple) as a first argument and the code as a second argument. +Each number in mask configures the amount of range shrink +1 - no shrink +2 - 2x shrink +3 - 3x shrink +.... +Up to 8x shrink. +[example:range_shrank] +Note: see mortonEncode() docs on why range change might be beneficial. +Still limited to 8 numbers at most. + +Morton code for one argument is always the argument itself (as a tuple). +[example:identity] +Produces: `(1)` + +You can shrink one argument too: +[example:identity_shrank] +Produces: `(128)` + +The function accepts a column of codes as a second argument: +[example:from_table] + +The range tuple must be a constant: +[example:from_table_range] +)", + Documentation::Examples{ + {"simple", "SELECT mortonDecode(4, 2149)"}, + {"range_shrank", "SELECT mortonDecode((1,2), 1572864)"}, + {"identity", "SELECT mortonDecode(1, 1)"}, + {"identity_shrank", "SELECT mortonDecode(tuple(2), 32768)"}, + {"from_table", "SELECT mortonDecode(2, code) FROM table"}, + {"from_table_range", "SELECT mortonDecode((1,2), code) FROM table"}, + }, + Documentation::Categories {"ZCurve", "Morton coding"} + }); +} + +} diff --git a/src/Functions/mortonEncode.cpp b/src/Functions/mortonEncode.cpp new file mode 100644 index 00000000000..4bdd237fa9c --- /dev/null +++ b/src/Functions/mortonEncode.cpp @@ -0,0 +1,393 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#if USE_MULTITARGET_CODE && defined(__BMI2__) +#include +#endif + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; +} + +#define EXTRACT_VECTOR(INDEX) \ + const ColumnPtr & col##INDEX = non_const_arguments[(INDEX) + vectorStartIndex].column; + +#define ENCODE(ND, ...) \ + if (nd == (ND)) \ + { \ + for (size_t i = 0; i < input_rows_count; i++) \ + { \ + vec_res[i] = MortonND_##ND##D_Enc.Encode(__VA_ARGS__); \ + } \ + return col_res; \ + } + +#define EXPAND(IDX, ...) \ + (mask) ? expand(mask->getColumn(IDX).getUInt(0), __VA_ARGS__) : __VA_ARGS__ + +#define MASK(ND, IDX, ...) \ + (EXPAND(IDX, __VA_ARGS__) & MortonND_##ND##D_Enc.InputMask()) + +#define EXECUTE() \ + size_t nd = arguments.size(); \ + size_t vectorStartIndex = 0; \ + const auto * const_col = typeid_cast(arguments[0].column.get()); \ + const ColumnTuple * mask; \ + if (const_col) \ + mask = typeid_cast(const_col->getDataColumnPtr().get()); \ + else \ + mask = typeid_cast(arguments[0].column.get()); \ + if (mask) \ + { \ + nd = mask->tupleSize(); \ + vectorStartIndex = 1; \ + for (size_t i = 0; i < nd; i++) \ + { \ + auto ratio = mask->getColumn(i).getUInt(0); \ + if (ratio > 8 || ratio < 1) \ + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, \ + "Illegal argument {} of function {}, should be a number in range 1-8", \ + arguments[0].column->getName(), getName()); \ + } \ + } \ + \ + auto non_const_arguments = arguments; \ + for (auto & argument : non_const_arguments) \ + argument.column = argument.column->convertToFullColumnIfConst(); \ + \ + auto col_res = ColumnUInt64::create(); \ + ColumnUInt64::Container & vec_res = col_res->getData(); \ + vec_res.resize(input_rows_count); \ + \ + EXTRACT_VECTOR(0) \ + if (nd == 1) \ + { \ + for (size_t i = 0; i < input_rows_count; i++) \ + { \ + vec_res[i] = EXPAND(0, col0->getUInt(i)); \ + } \ + return col_res; \ + } \ + \ + EXTRACT_VECTOR(1) \ + ENCODE(2, \ + MASK(2, 0, col0->getUInt(i)), \ + MASK(2, 1, col1->getUInt(i))) \ + EXTRACT_VECTOR(2) \ + ENCODE(3, \ + MASK(3, 0, col0->getUInt(i)), \ + MASK(3, 1, col1->getUInt(i)), \ + MASK(3, 2, col2->getUInt(i))) \ + EXTRACT_VECTOR(3) \ + ENCODE(4, \ + MASK(4, 0, col0->getUInt(i)), \ + MASK(4, 1, col1->getUInt(i)), \ + MASK(4, 2, col2->getUInt(i)), \ + MASK(4, 3, col3->getUInt(i))) \ + EXTRACT_VECTOR(4) \ + ENCODE(5, \ + MASK(5, 0, col0->getUInt(i)), \ + MASK(5, 1, col1->getUInt(i)), \ + MASK(5, 2, col2->getUInt(i)), \ + MASK(5, 3, col3->getUInt(i)), \ + MASK(5, 4, col4->getUInt(i))) \ + EXTRACT_VECTOR(5) \ + ENCODE(6, \ + MASK(6, 0, col0->getUInt(i)), \ + MASK(6, 1, col1->getUInt(i)), \ + MASK(6, 2, col2->getUInt(i)), \ + MASK(6, 3, col3->getUInt(i)), \ + MASK(6, 4, col4->getUInt(i)), \ + MASK(6, 5, col5->getUInt(i))) \ + EXTRACT_VECTOR(6) \ + ENCODE(7, \ + MASK(7, 0, col0->getUInt(i)), \ + MASK(7, 1, col1->getUInt(i)), \ + MASK(7, 2, col2->getUInt(i)), \ + MASK(7, 3, col3->getUInt(i)), \ + MASK(7, 4, col4->getUInt(i)), \ + MASK(7, 5, col5->getUInt(i)), \ + MASK(7, 6, col6->getUInt(i))) \ + EXTRACT_VECTOR(7) \ + ENCODE(8, \ + MASK(8, 0, col0->getUInt(i)), \ + MASK(8, 1, col1->getUInt(i)), \ + MASK(8, 2, col2->getUInt(i)), \ + MASK(8, 3, col3->getUInt(i)), \ + MASK(8, 4, col4->getUInt(i)), \ + MASK(8, 5, col5->getUInt(i)), \ + MASK(8, 6, col6->getUInt(i)), \ + MASK(8, 7, col7->getUInt(i))) \ + \ + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, \ + "Illegal number of UInt arguments of function {}, max: 8", \ + getName()); \ + +DECLARE_DEFAULT_CODE( +constexpr auto MortonND_2D_Enc = mortonnd::MortonNDLutEncoder<2, 32, 8>(); +constexpr auto MortonND_3D_Enc = mortonnd::MortonNDLutEncoder<3, 21, 8>(); +constexpr auto MortonND_4D_Enc = mortonnd::MortonNDLutEncoder<4, 16, 8>(); +constexpr auto MortonND_5D_Enc = mortonnd::MortonNDLutEncoder<5, 12, 8>(); +constexpr auto MortonND_6D_Enc = mortonnd::MortonNDLutEncoder<6, 10, 8>(); +constexpr auto MortonND_7D_Enc = mortonnd::MortonNDLutEncoder<7, 9, 8>(); +constexpr auto MortonND_8D_Enc = mortonnd::MortonNDLutEncoder<8, 8, 8>(); +class FunctionMortonEncode : public IFunction +{ +public: + static constexpr auto name = "mortonEncode"; + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool isVariadic() const override + { + return true; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DB::DataTypes & arguments) const override + { + size_t vectorStartIndex = 0; + if (arguments.empty()) + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, + "At least one UInt argument is required for function {}", + getName()); + if (WhichDataType(arguments[0]).isTuple()) + { + vectorStartIndex = 1; + const auto * type_tuple = typeid_cast(arguments[0].get()); + auto tuple_size = type_tuple->getElements().size(); + if (tuple_size != (arguments.size() - 1)) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Illegal argument {} for function {}, tuple size should be equal to number of UInt arguments", + arguments[0]->getName(), getName()); + for (size_t i = 0; i < tuple_size; i++) + { + if (!WhichDataType(type_tuple->getElement(i)).isNativeUInt()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument in tuple for function {}, should be a native UInt", + type_tuple->getElement(i)->getName(), getName()); + } + } + + for (size_t i = vectorStartIndex; i < arguments.size(); i++) + { + const auto & arg = arguments[i]; + if (!WhichDataType(arg).isNativeUInt()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}, should be a native UInt", + arg->getName(), getName()); + } + return std::make_shared(); + } + + static UInt64 expand(UInt64 ratio, UInt64 value) + { + switch (ratio) + { + case 1: + return value; + case 2: + return MortonND_2D_Enc.Encode(0, value & MortonND_2D_Enc.InputMask()); + case 3: + return MortonND_3D_Enc.Encode(0, 0, value & MortonND_3D_Enc.InputMask()); + case 4: + return MortonND_4D_Enc.Encode(0, 0, 0, value & MortonND_4D_Enc.InputMask()); + case 5: + return MortonND_5D_Enc.Encode(0, 0, 0, 0, value & MortonND_5D_Enc.InputMask()); + case 6: + return MortonND_6D_Enc.Encode(0, 0, 0, 0, 0, value & MortonND_6D_Enc.InputMask()); + case 7: + return MortonND_7D_Enc.Encode(0, 0, 0, 0, 0, 0, value & MortonND_7D_Enc.InputMask()); + case 8: + return MortonND_8D_Enc.Encode(0, 0, 0, 0, 0, 0, 0, value & MortonND_8D_Enc.InputMask()); + } + return value; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + EXECUTE() + } +}; +) // DECLARE_DEFAULT_CODE + +#if defined(MORTON_ND_BMI2_ENABLED) +#undef ENCODE +#define ENCODE(ND, ...) \ + if (nd == (ND)) \ + { \ + for (size_t i = 0; i < input_rows_count; i++) \ + { \ + vec_res[i] = MortonND_##ND##D::Encode(__VA_ARGS__); \ + } \ + return col_res; \ + } + +#undef MASK +#define MASK(ND, IDX, ...) \ + (EXPAND(IDX, __VA_ARGS__)) + +DECLARE_AVX2_SPECIFIC_CODE( +using MortonND_2D = mortonnd::MortonNDBmi<2, uint64_t>; +using MortonND_3D = mortonnd::MortonNDBmi<3, uint64_t>; +using MortonND_4D = mortonnd::MortonNDBmi<4, uint64_t>; +using MortonND_5D = mortonnd::MortonNDBmi<5, uint64_t>; +using MortonND_6D = mortonnd::MortonNDBmi<6, uint64_t>; +using MortonND_7D = mortonnd::MortonNDBmi<7, uint64_t>; +using MortonND_8D = mortonnd::MortonNDBmi<8, uint64_t>; + +class FunctionMortonEncode : public TargetSpecific::Default::FunctionMortonEncode +{ +public: + static UInt64 expand(UInt64 ratio, UInt64 value) + { + switch (ratio) + { + case 1: + return value; + case 2: + return MortonND_2D::Encode(0, value); + case 3: + return MortonND_3D::Encode(0, 0, value); + case 4: + return MortonND_4D::Encode(0, 0, 0, value); + case 5: + return MortonND_5D::Encode(0, 0, 0, 0, value); + case 6: + return MortonND_6D::Encode(0, 0, 0, 0, 0, value); + case 7: + return MortonND_7D::Encode(0, 0, 0, 0, 0, 0, value); + case 8: + return MortonND_8D::Encode(0, 0, 0, 0, 0, 0, 0, value); + } + return value; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + EXECUTE() + } +}; +) // DECLARE_AVX2_SPECIFIC_CODE +#endif // MORTON_ND_BMI2_ENABLED + +#undef ENCODE +#undef MASK +#undef EXTRACT_VECTOR +#undef EXPAND +#undef EXECUTE + +class FunctionMortonEncode: public TargetSpecific::Default::FunctionMortonEncode +{ +public: + explicit FunctionMortonEncode(ContextPtr context) : selector(context) + { + selector.registerImplementation(); + +#if USE_MULTITARGET_CODE && defined(MORTON_ND_BMI2_ENABLED) + selector.registerImplementation(); +#endif + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + return selector.selectAndExecute(arguments, result_type, input_rows_count); + } + + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context); + } + +private: + ImplementationSelector selector; +}; + +REGISTER_FUNCTION(MortonEncode) +{ + factory.registerFunction({ + R"( +Calculates Morton encoding (ZCurve) for a list of unsigned integers + +The function has two modes of operation: +- Simple +- Expanded + +Simple: accepts up to 8 unsigned integers as arguments and produces a UInt64 code. +[example:simple] + +Expanded: accepts a range mask (tuple) as a first argument and up to 8 unsigned integers as other arguments. +Each number in mask configures the amount of range expansion +1 - no expansion +2 - 2x expansion +3 - 3x expansion +.... +Up to 8x expansion. +[example:range_expanded] +Note: tuple size must be equal to the number of the other arguments + +Range expansion can be beneficial when you need a similar distribution for arguments with wildly different ranges (or cardinality) +For example: 'IP Address' (0...FFFFFFFF) and 'Country code' (0...FF) + +Morton encoding for one argument is always the argument itself. +[example:identity] +Produces: `1` + +You can expand one argument too: +[example:identity_expanded] +Produces: `32768` + +The function also accepts columns as arguments: +[example:from_table] + +But the range tuple must still be a constant: +[example:from_table_range] + +Please note that you can fit only so much bits of information into Morton code as UInt64 has. +Two arguments will have a range of maximum 2^32 (64/2) each +Three arguments: range of max 2^21 (64/3) each +And so on, all overflow will be clamped to zero +)", + Documentation::Examples{ + {"simple", "SELECT mortonEncode(1, 2, 3)"}, + {"range_expanded", "SELECT mortonEncode((1,2), 1024, 16)"}, + {"identity", "SELECT mortonEncode(1)"}, + {"identity_expanded", "SELECT mortonEncode(tuple(2), 128)"}, + {"from_table", "SELECT mortonEncode(n1, n2) FROM table"}, + {"from_table_range", "SELECT mortonEncode((1,2), n1, n2) FROM table"}, + }, + Documentation::Categories {"ZCurve", "Morton coding"} + }); +} + +} diff --git a/tests/queries/0_stateless/02457_morton_coding.reference b/tests/queries/0_stateless/02457_morton_coding.reference new file mode 100644 index 00000000000..311a515a458 --- /dev/null +++ b/tests/queries/0_stateless/02457_morton_coding.reference @@ -0,0 +1,12 @@ +----- START ----- +----- CONST ----- +2149 +(1,2,3,4) +4294967286 +(65534,65533) +4294967286 +(4294967286) +----- 256, 8 ----- +----- 65536, 4 ----- +----- 4294967296, 2 ----- +----- END ----- diff --git a/tests/queries/0_stateless/02457_morton_coding.sql b/tests/queries/0_stateless/02457_morton_coding.sql new file mode 100644 index 00000000000..4fc26f255f4 --- /dev/null +++ b/tests/queries/0_stateless/02457_morton_coding.sql @@ -0,0 +1,137 @@ +SELECT '----- START -----'; +drop table if exists morton_numbers_02457; +create table morton_numbers_02457( + n1 UInt32, + n2 UInt32, + n3 UInt16, + n4 UInt16, + n5 UInt8, + n6 UInt8, + n7 UInt8, + n8 UInt8 +) + Engine=MergeTree() + ORDER BY n1; + +SELECT '----- CONST -----'; +select mortonEncode(1,2,3,4); +select mortonDecode(4, 2149); +select mortonEncode(65534, 65533); +select mortonDecode(2, 4294967286); +select mortonEncode(4294967286); +select mortonDecode(1, 4294967286); + +SELECT '----- 256, 8 -----'; +insert into morton_numbers_02457 +select n1.number, n2.number, n3.number, n4.number, n5.number, n6.number, n7.number, n8.number +from numbers(256-4, 4) n1 + cross join numbers(256-4, 4) n2 + cross join numbers(256-4, 4) n3 + cross join numbers(256-4, 4) n4 + cross join numbers(256-4, 4) n5 + cross join numbers(256-4, 4) n6 + cross join numbers(256-4, 4) n7 + cross join numbers(256-4, 4) n8 +; +drop table if exists morton_numbers_1_02457; +create table morton_numbers_1_02457( + n1 UInt64, + n2 UInt64, + n3 UInt64, + n4 UInt64, + n5 UInt64, + n6 UInt64, + n7 UInt64, + n8 UInt64 +) + Engine=MergeTree() + ORDER BY n1; + +insert into morton_numbers_1_02457 +select untuple(mortonDecode(8, mortonEncode(n1, n2, n3, n4, n5, n6, n7, n8))) +from morton_numbers_02457; + +( + select * from morton_numbers_02457 + union distinct + select * from morton_numbers_1_02457 +) +except +( + select * from morton_numbers_02457 + intersect + select * from morton_numbers_1_02457 +); +drop table if exists morton_numbers_1_02457; + +SELECT '----- 65536, 4 -----'; +insert into morton_numbers_02457 +select n1.number, n2.number, n3.number, n4.number, 0, 0, 0, 0 +from numbers(pow(2, 16)-8,8) n1 + cross join numbers(pow(2, 16)-8, 8) n2 + cross join numbers(pow(2, 16)-8, 8) n3 + cross join numbers(pow(2, 16)-8, 8) n4 +; + +create table morton_numbers_2_02457( + n1 UInt64, + n2 UInt64, + n3 UInt64, + n4 UInt64 +) + Engine=MergeTree() + ORDER BY n1; + +insert into morton_numbers_2_02457 +select untuple(mortonDecode(4, mortonEncode(n1, n2, n3, n4))) +from morton_numbers_02457; + +( + select n1, n2, n3, n4 from morton_numbers_02457 + union distinct + select n1, n2, n3, n4 from morton_numbers_2_02457 +) +except +( + select n1, n2, n3, n4 from morton_numbers_02457 + intersect + select n1, n2, n3, n4 from morton_numbers_2_02457 +); +drop table if exists morton_numbers_2_02457; + +SELECT '----- 4294967296, 2 -----'; +insert into morton_numbers_02457 +select n1.number, n2.number, 0, 0, 0, 0, 0, 0 +from numbers(pow(2, 32)-8,8) n1 + cross join numbers(pow(2, 32)-8, 8) n2 + cross join numbers(pow(2, 32)-8, 8) n3 + cross join numbers(pow(2, 32)-8, 8) n4 +; + +drop table if exists morton_numbers_3_02457; +create table morton_numbers_3_02457( + n1 UInt64, + n2 UInt64 +) + Engine=MergeTree() + ORDER BY n1; + +insert into morton_numbers_3_02457 +select untuple(mortonDecode(2, mortonEncode(n1, n2))) +from morton_numbers_02457; + +( + select n1, n2 from morton_numbers_3_02457 + union distinct + select n1, n2 from morton_numbers_3_02457 +) +except +( + select n1, n2 from morton_numbers_3_02457 + intersect + select n1, n2 from morton_numbers_3_02457 +); +drop table if exists morton_numbers_3_02457; + +SELECT '----- END -----'; +drop table if exists morton_numbers_02457; diff --git a/tests/queries/0_stateless/02457_morton_coding_with_mask.reference b/tests/queries/0_stateless/02457_morton_coding_with_mask.reference new file mode 100644 index 00000000000..32d5ce3ee27 --- /dev/null +++ b/tests/queries/0_stateless/02457_morton_coding_with_mask.reference @@ -0,0 +1,15 @@ +----- START ----- +----- CONST ----- +4205569 +(1,2,3,4) +4294967286 +(65534,65533) +4294967286 +(4294967286) +2147483648 +(128) +0 +----- (1,2,1,2) ----- +----- (1,4) ----- +----- (1,1,2) ----- +----- END ----- diff --git a/tests/queries/0_stateless/02457_morton_coding_with_mask.sql b/tests/queries/0_stateless/02457_morton_coding_with_mask.sql new file mode 100644 index 00000000000..5aeb1f380be --- /dev/null +++ b/tests/queries/0_stateless/02457_morton_coding_with_mask.sql @@ -0,0 +1,143 @@ +SELECT '----- START -----'; + +SELECT '----- CONST -----'; +select mortonEncode((1,2,3,1), 1,2,3,4); +select mortonDecode((1, 2, 3, 1), 4205569); +select mortonEncode((1,1), 65534, 65533); +select mortonDecode((1,1), 4294967286); +select mortonEncode(tuple(1), 4294967286); +select mortonDecode(tuple(1), 4294967286); +select mortonEncode(tuple(4), 128); +select mortonDecode(tuple(4), 2147483648); +select mortonEncode((4,4,4,4), 128, 128, 128, 128); + +SELECT '----- (1,2,1,2) -----'; +drop table if exists morton_numbers_mask_02457; +create table morton_numbers_mask_02457( + n1 UInt8, + n2 UInt8, + n3 UInt8, + n4 UInt8 +) + Engine=MergeTree() + ORDER BY n1; + +insert into morton_numbers_mask_02457 +select n1.number, n2.number, n3.number, n4.number +from numbers(256-16, 16) n1 + cross join numbers(256-16, 16) n2 + cross join numbers(256-16, 16) n3 + cross join numbers(256-16, 16) n4 +; +drop table if exists morton_numbers_mask_1_02457; +create table morton_numbers_mask_1_02457( + n1 UInt64, + n2 UInt64, + n3 UInt64, + n4 UInt64 +) + Engine=MergeTree() + ORDER BY n1; + +insert into morton_numbers_mask_1_02457 +select untuple(mortonDecode((1,2,1,2), mortonEncode((1,2,1,2), n1, n2, n3, n4))) +from morton_numbers_mask_02457; + +( + select * from morton_numbers_mask_02457 + union distinct + select * from morton_numbers_mask_1_02457 +) +except +( + select * from morton_numbers_mask_02457 + intersect + select * from morton_numbers_mask_1_02457 +); +drop table if exists morton_numbers_mask_02457; +drop table if exists morton_numbers_mask_1_02457; + +SELECT '----- (1,4) -----'; +drop table if exists morton_numbers_mask_02457; +create table morton_numbers_mask_02457( + n1 UInt32, + n2 UInt8 +) + Engine=MergeTree() + ORDER BY n1; + +insert into morton_numbers_mask_02457 +select n1.number, n2.number +from numbers(pow(2, 32)-64, 64) n1 + cross join numbers(pow(2, 8)-64, 64) n2 +; +drop table if exists morton_numbers_mask_2_02457; +create table morton_numbers_mask_2_02457( + n1 UInt64, + n2 UInt64 +) + Engine=MergeTree() + ORDER BY n1; + +insert into morton_numbers_mask_2_02457 +select untuple(mortonDecode((1,4), mortonEncode((1,4), n1, n2))) +from morton_numbers_mask_02457; + +( + select * from morton_numbers_mask_02457 + union distinct + select * from morton_numbers_mask_2_02457 +) +except +( + select * from morton_numbers_mask_02457 + intersect + select * from morton_numbers_mask_2_02457 +); +drop table if exists morton_numbers_mask_02457; +drop table if exists morton_numbers_mask_2_02457; + +SELECT '----- (1,1,2) -----'; +drop table if exists morton_numbers_mask_02457; +create table morton_numbers_mask_02457( + n1 UInt16, + n2 UInt16, + n3 UInt8, +) + Engine=MergeTree() + ORDER BY n1; + +insert into morton_numbers_mask_02457 +select n1.number, n2.number, n3.number +from numbers(pow(2, 16)-64, 64) n1 + cross join numbers(pow(2, 16)-64, 64) n2 + cross join numbers(pow(2, 8)-64, 64) n3 +; +drop table if exists morton_numbers_mask_3_02457; +create table morton_numbers_mask_3_02457( + n1 UInt64, + n2 UInt64, + n3 UInt64 +) + Engine=MergeTree() + ORDER BY n1; + +insert into morton_numbers_mask_3_02457 +select untuple(mortonDecode((1,1,2), mortonEncode((1,1,2), n1, n2, n3))) +from morton_numbers_mask_02457; + +( + select * from morton_numbers_mask_02457 + union distinct + select * from morton_numbers_mask_3_02457 +) +except +( + select * from morton_numbers_mask_02457 + intersect + select * from morton_numbers_mask_3_02457 +); +drop table if exists morton_numbers_mask_02457; +drop table if exists morton_numbers_mask_3_02457; + +SELECT '----- END -----'; From f485c32317b49e7c5cad9518136e6d33e3d4f103 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 19 Oct 2022 23:38:44 +0000 Subject: [PATCH 038/188] Small improvements --- src/Parsers/ExpressionListParsers.cpp | 51 ++++++++++++++------------- 1 file changed, 26 insertions(+), 25 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index f7a016a59e4..1e3ffaa1c72 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -478,7 +478,10 @@ struct Operator { Operator() = default; - Operator(const std::string & function_name_, int priority_, int arity_ = 2, OperatorType type_ = OperatorType::None) + Operator(const std::string & function_name_, + int priority_, + int arity_, + OperatorType type_ = OperatorType::None) : type(type_), priority(priority_), arity(arity_), function_name(function_name_) {} OperatorType type; @@ -507,9 +510,7 @@ class Layer { public: explicit Layer(bool allow_alias_ = true, bool allow_alias_without_as_keyword_ = true) : - allow_alias(allow_alias_), allow_alias_without_as_keyword(allow_alias_without_as_keyword_) - { - } + allow_alias(allow_alias_), allow_alias_without_as_keyword(allow_alias_without_as_keyword_) {} virtual ~Layer() = default; @@ -810,10 +811,10 @@ public: }; -class OrdinaryFunctionLayer : public Layer +class FunctionLayer : public Layer { public: - explicit OrdinaryFunctionLayer(String function_name_, bool allow_function_parameters_ = true) + explicit FunctionLayer(String function_name_, bool allow_function_parameters_ = true) : function_name(function_name_), allow_function_parameters(allow_function_parameters_){} bool parse(IParser::Pos & pos, Expected & expected, Action & action) override @@ -958,7 +959,7 @@ public: if (parameters) { - function_node->parameters = parameters; + function_node->parameters = std::move(parameters); function_node->children.push_back(function_node->parameters); } @@ -991,7 +992,7 @@ public: return false; } - elements = {function_node}; + elements = {std::move(function_node)}; finished = true; } @@ -1990,9 +1991,9 @@ std::unique_ptr getFunctionLayer(ASTPtr identifier, bool is_table_functio || function_name_lowercase == "timestampdiff" || function_name_lowercase == "timestamp_diff") return std::make_unique(); else if (function_name_lowercase == "grouping") - return std::make_unique(function_name_lowercase, allow_function_parameters_); + return std::make_unique(function_name_lowercase, allow_function_parameters_); else - return std::make_unique(function_name, allow_function_parameters_); + return std::make_unique(function_name, allow_function_parameters_); } @@ -2141,22 +2142,22 @@ std::vector> ParserExpressionImpl::operators_t {"<", Operator("less", 9, 2, OperatorType::Comparison)}, {">", Operator("greater", 9, 2, OperatorType::Comparison)}, {"=", Operator("equals", 9, 2, OperatorType::Comparison)}, - {"LIKE", Operator("like", 9)}, - {"ILIKE", Operator("ilike", 9)}, - {"NOT LIKE", Operator("notLike", 9)}, - {"NOT ILIKE", Operator("notILike", 9)}, - {"IN", Operator("in", 9)}, - {"NOT IN", Operator("notIn", 9)}, - {"GLOBAL IN", Operator("globalIn", 9)}, - {"GLOBAL NOT IN", Operator("globalNotIn", 9)}, + {"LIKE", Operator("like", 9, 2)}, + {"ILIKE", Operator("ilike", 9, 2)}, + {"NOT LIKE", Operator("notLike", 9, 2)}, + {"NOT ILIKE", Operator("notILike", 9, 2)}, + {"IN", Operator("in", 9, 2)}, + {"NOT IN", Operator("notIn", 9, 2)}, + {"GLOBAL IN", Operator("globalIn", 9, 2)}, + {"GLOBAL NOT IN", Operator("globalNotIn", 9, 2)}, {"||", Operator("concat", 10, 2, OperatorType::Mergeable)}, - {"+", Operator("plus", 11)}, - {"-", Operator("minus", 11)}, - {"*", Operator("multiply", 12)}, - {"/", Operator("divide", 12)}, - {"%", Operator("modulo", 12)}, - {"MOD", Operator("modulo", 12)}, - {"DIV", Operator("intDiv", 12)}, + {"+", Operator("plus", 11, 2)}, + {"-", Operator("minus", 11, 2)}, + {"*", Operator("multiply", 12, 2)}, + {"/", Operator("divide", 12, 2)}, + {"%", Operator("modulo", 12, 2)}, + {"MOD", Operator("modulo", 12, 2)}, + {"DIV", Operator("intDiv", 12, 2)}, {".", Operator("tupleElement", 14, 2, OperatorType::TupleElement)}, {"[", Operator("arrayElement", 14, 2, OperatorType::ArrayElement)}, {"::", Operator("CAST", 14, 2, OperatorType::Cast)}, From 159a251f789fd7aaed23168ba40149f0b9856ab7 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 19 Oct 2022 23:39:35 +0000 Subject: [PATCH 039/188] Fix aliases --- src/Parsers/ExpressionListParsers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 1e3ffaa1c72..fbdfa597f82 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -509,7 +509,7 @@ enum class Checkpoint class Layer { public: - explicit Layer(bool allow_alias_ = true, bool allow_alias_without_as_keyword_ = true) : + explicit Layer(bool allow_alias_ = true, bool allow_alias_without_as_keyword_ = false) : allow_alias(allow_alias_), allow_alias_without_as_keyword(allow_alias_without_as_keyword_) {} virtual ~Layer() = default; From 57ffcb7ed06bfe65d020457a2a8377c6f4d15556 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Oct 2022 19:51:27 +0200 Subject: [PATCH 040/188] Revert "Merge pull request #41681 from ClickHouse/revert-41268-disable-s3-parallel-write-for-part-moves-to-disk-s3" This reverts commit 7956c2becf8d81f5d73b6f8f5e6d0ff1f05224a7, reversing changes made to 57be648984d0255669a0d9ffb16a7c04fa79217c. --- src/Core/Settings.h | 1 + src/Disks/IDisk.cpp | 19 ++++--- src/Disks/IDisk.h | 6 ++- .../ObjectStorages/S3/S3ObjectStorage.cpp | 4 +- src/IO/WriteSettings.h | 1 + src/Interpreters/Context.cpp | 1 + .../integration/test_s3_zero_copy_ttl/test.py | 54 +++++++++++++++++++ 7 files changed, 77 insertions(+), 9 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 07618ee731d..5d4b3c823a9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -93,6 +93,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ + M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 3704a511478..8a6bea2565b 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -24,13 +24,13 @@ bool IDisk::isDirectoryEmpty(const String & path) const return !iterateDirectory(path)->isValid(); } -void IDisk::copyFile(const String & from_file_path, IDisk & to_disk, const String & to_file_path) +void IDisk::copyFile(const String & from_file_path, IDisk & to_disk, const String & to_file_path, const WriteSettings & settings) /// NOLINT { LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} (path: {}) {} to {} (path: {}) {}.", getName(), getPath(), from_file_path, to_disk.getName(), to_disk.getPath(), to_file_path); auto in = readFile(from_file_path); - auto out = to_disk.writeFile(to_file_path); + auto out = to_disk.writeFile(to_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, settings); copyData(*in, *out); out->finalize(); } @@ -56,15 +56,15 @@ void IDisk::removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_ba using ResultsCollector = std::vector>; -void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_path, Executor & exec, ResultsCollector & results, bool copy_root_dir) +void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_path, Executor & exec, ResultsCollector & results, bool copy_root_dir, const WriteSettings & settings) { if (from_disk.isFile(from_path)) { auto result = exec.execute( - [&from_disk, from_path, &to_disk, to_path]() + [&from_disk, from_path, &to_disk, to_path, &settings]() { setThreadName("DiskCopier"); - from_disk.copyFile(from_path, to_disk, fs::path(to_path) / fileName(from_path)); + from_disk.copyFile(from_path, to_disk, fs::path(to_path) / fileName(from_path), settings); }); results.push_back(std::move(result)); @@ -80,7 +80,7 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p } for (auto it = from_disk.iterateDirectory(from_path); it->isValid(); it->next()) - asyncCopy(from_disk, it->path(), to_disk, dest, exec, results, true); + asyncCopy(from_disk, it->path(), to_disk, dest, exec, results, true, settings); } } @@ -89,7 +89,12 @@ void IDisk::copyThroughBuffers(const String & from_path, const std::shared_ptrgetExecutor(); ResultsCollector results; - asyncCopy(*this, from_path, *to_disk, to_path, exec, results, copy_root_dir); + WriteSettings settings; + /// Disable parallel write. We already copy in parallel. + /// Avoid high memory usage. See test_s3_zero_copy_ttl/test.py::test_move_and_s3_memory_usage + settings.s3_allow_parallel_part_upload = false; + + asyncCopy(*this, from_path, *to_disk, to_path, exec, results, copy_root_dir, settings); for (auto & result : results) result.wait(); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 4a7be740ccf..66a5c55f7f7 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -181,7 +181,11 @@ public: virtual void copyDirectoryContent(const String & from_dir, const std::shared_ptr & to_disk, const String & to_dir); /// Copy file `from_file_path` to `to_file_path` located at `to_disk`. - virtual void copyFile(const String & from_file_path, IDisk & to_disk, const String & to_file_path); + virtual void copyFile( /// NOLINT + const String & from_file_path, + IDisk & to_disk, + const String & to_file_path, + const WriteSettings & settings = {}); /// List files at `path` and add their names to `file_names` virtual void listFiles(const String & path, std::vector & file_names) const = 0; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 213f744d84f..33da39b5813 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -230,7 +230,9 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files"); auto settings_ptr = s3_settings.get(); - auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "VFSWrite"); + ScheduleFunc scheduler; + if (write_settings.s3_allow_parallel_part_upload) + scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "VFSWrite"); auto s3_buffer = std::make_unique( client.get(), diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 38a706997cf..a1f5b23fb97 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -15,6 +15,7 @@ struct WriteSettings bool enable_filesystem_cache_on_write_operations = false; bool enable_filesystem_cache_log = false; bool is_file_cache_persistent = false; + bool s3_allow_parallel_part_upload = true; /// Monitoring bool for_object_storage = false; // to choose which profile events should be incremented diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1de56e950c6..98c782e2acc 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3634,6 +3634,7 @@ WriteSettings Context::getWriteSettings() const res.enable_filesystem_cache_on_write_operations = settings.enable_filesystem_cache_on_write_operations; res.enable_filesystem_cache_log = settings.enable_filesystem_cache_log; + res.s3_allow_parallel_part_upload = settings.s3_allow_parallel_part_upload; res.remote_throttler = getRemoteWriteThrottler(); diff --git a/tests/integration/test_s3_zero_copy_ttl/test.py b/tests/integration/test_s3_zero_copy_ttl/test.py index 9a782aacef6..843c216ce72 100644 --- a/tests/integration/test_s3_zero_copy_ttl/test.py +++ b/tests/integration/test_s3_zero_copy_ttl/test.py @@ -15,6 +15,11 @@ node3 = cluster.add_instance( "node3", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True ) +single_node_cluster = ClickHouseCluster(__file__) +small_node = single_node_cluster.add_instance( + "small_node", main_configs=["configs/s3.xml"], with_minio=True +) + @pytest.fixture(scope="module") def started_cluster(): @@ -92,3 +97,52 @@ def test_ttl_move_and_s3(started_cluster): print(f"Attempts remaining: {attempt}") assert counter == 300 + + +@pytest.fixture(scope="module") +def started_single_node_cluster(): + try: + single_node_cluster.start() + + yield single_node_cluster + finally: + single_node_cluster.shutdown() + + +def test_move_and_s3_memory_usage(started_single_node_cluster): + if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): + pytest.skip("Disabled for debug and sanitizers. Too slow.") + + small_node.query( + "CREATE TABLE s3_test_with_ttl (x UInt32, a String codec(NONE), b String codec(NONE), c String codec(NONE), d String codec(NONE), e String codec(NONE)) engine = MergeTree order by x partition by x SETTINGS storage_policy='s3_and_default'" + ) + + for _ in range(10): + small_node.query( + "insert into s3_test_with_ttl select 0, repeat('a', 100), repeat('b', 100), repeat('c', 100), repeat('d', 100), repeat('e', 100) from zeros(400000) settings max_block_size = 8192, max_insert_block_size=10000000, min_insert_block_size_rows=10000000" + ) + + # After this, we should have 5 columns per 10 * 100 * 400000 ~ 400 MB; total ~2G data in partition + small_node.query("optimize table s3_test_with_ttl final") + + small_node.query("system flush logs") + # Will take memory usage from metric_log. + # It is easier then specifying total memory limit (insert queries can hit this limit). + small_node.query("truncate table system.metric_log") + + small_node.query( + "alter table s3_test_with_ttl move partition 0 to volume 'external'", + settings={"send_logs_level": "error"}, + ) + small_node.query("system flush logs") + max_usage = small_node.query( + "select max(CurrentMetric_MemoryTracking) from system.metric_log" + ) + # 3G limit is a big one. However, we can hit it anyway with parallel s3 writes enabled. + # Also actual value can be bigger because of memory drift. + # Increase it a little bit if test fails. + assert int(max_usage) < 3e9 + res = small_node.query( + "select * from system.errors where last_error_message like '%Memory limit%' limit 1" + ) + assert res == "" From fb67a2f2ce8401288a86b0eba5fb3d9a3c6500a2 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 21 Oct 2022 00:16:24 +0000 Subject: [PATCH 041/188] Fixes & improvements --- src/Parsers/ExpressionListParsers.cpp | 210 ++++++++++++++------------ 1 file changed, 114 insertions(+), 96 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index fbdfa597f82..a38d7ab0c98 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -613,13 +613,17 @@ public: /// bool mergeElement(bool push_to_elements = true) { + parsed_alias = false; + Operator cur_op; while (popOperator(cur_op)) { ASTPtr function; - // Special case of ternary operator - if (cur_op.type == OperatorType::StartIf) + // We should not meet the starting part of the operator while finishing an element + if (cur_op.type == OperatorType::StartIf || + cur_op.type == OperatorType::StartBetween || + cur_op.type == OperatorType::StartNotBetween) return false; if (cur_op.type == OperatorType::FinishIf) @@ -629,10 +633,6 @@ public: return false; } - // Special case of a BETWEEN b AND c operator - if (cur_op.type == OperatorType::StartBetween || cur_op.type == OperatorType::StartNotBetween) - return false; - if (cur_op.type == OperatorType::FinishBetween) { Operator tmp_op; @@ -728,6 +728,9 @@ public: /// In order to distinguish them we keep a counter of BETWEENs without matching ANDs. int between_counter = 0; + /// Flag we set when we parsed alias to avoid parsing next element as alias + bool parsed_alias = false; + bool allow_alias = true; bool allow_alias_without_as_keyword = true; @@ -777,7 +780,6 @@ public: } }; - /// Basic layer for a function with certain separator and end tokens: /// 1. If we parse a separator we should merge current operands and operators /// into one element and push in to 'elements' vector. @@ -787,6 +789,9 @@ template class BaseLayer : public Layer { public: + explicit BaseLayer(bool allow_alias_ = true, bool allow_alias_without_as_keyword_ = false) : + Layer(allow_alias_, allow_alias_without_as_keyword_) {} + bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { if (ParserToken(separator).ignore(pos, expected)) @@ -810,7 +815,7 @@ public: } }; - +/// Layer for regular and aggregate functions without syntax sugar class FunctionLayer : public Layer { public: @@ -1091,6 +1096,8 @@ public: class CastLayer : public Layer { public: + CastLayer() : Layer(true, true) {} + bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { /// CAST(x [AS alias1], T [AS alias2]) or CAST(x [AS alias1] AS T) @@ -1189,6 +1196,8 @@ public: class ExtractLayer : public BaseLayer { public: + ExtractLayer() : BaseLayer(true, true) {} + bool getResult(ASTPtr & node) override { if (state == 2) @@ -1258,6 +1267,8 @@ private: class SubstringLayer : public Layer { public: + SubstringLayer() : Layer(true, true) {} + bool getResult(ASTPtr & node) override { node = makeASTFunction("substring", std::move(elements)); @@ -1318,6 +1329,8 @@ public: class PositionLayer : public Layer { public: + PositionLayer() : Layer(true, true) {} + bool getResult(ASTPtr & node) override { if (state == 2) @@ -1383,10 +1396,11 @@ public: } }; - class ExistsLayer : public Layer { public: + ExistsLayer() : Layer(true, true) {} + bool parse(IParser::Pos & pos, Expected & expected, Action & /*action*/) override { ASTPtr node; @@ -1411,9 +1425,8 @@ public: class TrimLayer : public Layer { public: - TrimLayer(bool trim_left_, bool trim_right_) : trim_left(trim_left_), trim_right(trim_right_) - { - } + TrimLayer(bool trim_left_, bool trim_right_) + : Layer(true, true), trim_left(trim_left_), trim_right(trim_right_) {} bool getResult(ASTPtr & node) override { @@ -1571,13 +1584,11 @@ private: String function_name; }; - class DateAddLayer : public BaseLayer { public: - explicit DateAddLayer(const char * function_name_) : function_name(function_name_) - { - } + explicit DateAddLayer(const char * function_name_) + : BaseLayer(true, true), function_name(function_name_) {} bool getResult(ASTPtr & node) override { @@ -1628,10 +1639,11 @@ private: bool parsed_interval_kind = false; }; - class DateDiffLayer : public BaseLayer { public: + DateDiffLayer() : BaseLayer(true, true) {} + bool getResult(ASTPtr & node) override { if (parsed_interval_kind) @@ -1681,10 +1693,11 @@ private: bool parsed_interval_kind = false; }; - class IntervalLayer : public Layer { public: + IntervalLayer() : Layer(true, true) {} + bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { /// INTERVAL 1 HOUR or INTERVAL expr HOUR @@ -1759,86 +1772,11 @@ private: IntervalKind interval_kind; }; -/// Layer for table function 'view' and 'viewIfPermitted' -class ViewLayer : public Layer -{ -public: - explicit ViewLayer(bool if_permitted_) : if_permitted(if_permitted_) {} - - bool getResult(ASTPtr & node) override - { - if (if_permitted) - node = makeASTFunction("viewIfPermitted", std::move(elements)); - else - node = makeASTFunction("view", std::move(elements)); - - return true; - } - - bool parse(IParser::Pos & pos, Expected & expected, Action & /*action*/) override - { - /// view(SELECT ...) - /// viewIfPermitted(SELECT ... ELSE func(...)) - /// - /// 0. Parse the SELECT query and if 'if_permitted' parse 'ELSE' keyword (-> 1) else (finished) - /// 1. Parse closing token - - if (state == 0) - { - ASTPtr query; - - bool maybe_an_subquery = pos->type == TokenType::OpeningRoundBracket; - - if (!ParserSelectWithUnionQuery().parse(pos, query, expected)) - return false; - - auto & select_ast = query->as(); - if (select_ast.list_of_selects->children.size() == 1 && maybe_an_subquery) - { - // It's an subquery. Bail out. - return false; - } - - pushResult(query); - - if (!if_permitted) - { - if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) - return false; - - finished = true; - return true; - } - - if (!ParserKeyword{"ELSE"}.ignore(pos, expected)) - return false; - - state = 1; - return true; - } - - if (state == 1) - { - if (ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) - { - if (!mergeElement()) - return false; - - finished = true; - } - } - - return true; - } - -private: - bool if_permitted; -}; - - class CaseLayer : public Layer { public: + CaseLayer() : Layer(true, true) {} + bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { /// CASE [x] WHEN expr THEN expr [WHEN expr THEN expr [...]] [ELSE expr] END @@ -1927,6 +1865,82 @@ private: bool has_case_expr; }; +/// Layer for table function 'view' and 'viewIfPermitted' +class ViewLayer : public Layer +{ +public: + explicit ViewLayer(bool if_permitted_) : if_permitted(if_permitted_) {} + + bool getResult(ASTPtr & node) override + { + if (if_permitted) + node = makeASTFunction("viewIfPermitted", std::move(elements)); + else + node = makeASTFunction("view", std::move(elements)); + + return true; + } + + bool parse(IParser::Pos & pos, Expected & expected, Action & /*action*/) override + { + /// view(SELECT ...) + /// viewIfPermitted(SELECT ... ELSE func(...)) + /// + /// 0. Parse the SELECT query and if 'if_permitted' parse 'ELSE' keyword (-> 1) else (finished) + /// 1. Parse closing token + + if (state == 0) + { + ASTPtr query; + + bool maybe_an_subquery = pos->type == TokenType::OpeningRoundBracket; + + if (!ParserSelectWithUnionQuery().parse(pos, query, expected)) + return false; + + auto & select_ast = query->as(); + if (select_ast.list_of_selects->children.size() == 1 && maybe_an_subquery) + { + // It's an subquery. Bail out. + return false; + } + + pushResult(query); + + if (!if_permitted) + { + if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + return false; + + finished = true; + return true; + } + + if (!ParserKeyword{"ELSE"}.ignore(pos, expected)) + return false; + + state = 1; + return true; + } + + if (state == 1) + { + if (ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + { + if (!mergeElement()) + return false; + + finished = true; + } + } + + return true; + } + +private: + bool if_permitted; +}; + std::unique_ptr getFunctionLayer(ASTPtr identifier, bool is_table_function, bool allow_function_parameters_ = true) { @@ -2428,11 +2442,15 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po if (cur_op == operators_table.end()) { + ParserAlias alias_parser(layers.back()->allow_alias_without_as_keyword); auto old_pos = pos; - if (layers.back()->allow_alias && ParserAlias(layers.back()->allow_alias_without_as_keyword).parse(pos, tmp, expected)) + if (!layers.back()->parsed_alias && layers.back()->allow_alias && alias_parser.parse(pos, tmp, expected)) { if (layers.back()->insertAlias(tmp)) + { + layers.back()->parsed_alias = true; return Action::OPERATOR; + } } pos = old_pos; return Action::NONE; From 2033ea12169e5d511e32acb6b951606bb275f36a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 21 Oct 2022 14:53:23 +0200 Subject: [PATCH 042/188] Support subqueries in row policy filters. --- src/Interpreters/InterpreterSelectQuery.cpp | 11 ++++++--- tests/integration/test_row_policy/test.py | 27 +++++++++++++++++++++ 2 files changed, 34 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 79deb38317c..4139685c379 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -117,7 +117,8 @@ FilterDAGInfoPtr generateFilterActions( const StoragePtr & storage, const StorageSnapshotPtr & storage_snapshot, const StorageMetadataPtr & metadata_snapshot, - Names & prerequisite_columns) + Names & prerequisite_columns, + PreparedSetsPtr prepared_sets) { auto filter_info = std::make_shared(); @@ -155,7 +156,7 @@ FilterDAGInfoPtr generateFilterActions( /// Using separate expression analyzer to prevent any possible alias injection auto syntax_result = TreeRewriter(context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, storage_snapshot)); - SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot); + SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot, {}, false, {}, prepared_sets); filter_info->actions = analyzer.simpleSelectActions(); filter_info->column_name = expr_list->children.at(0)->getColumnName(); @@ -615,7 +616,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (row_policy_filter) { filter_info = generateFilterActions( - table_id, row_policy_filter, context, storage, storage_snapshot, metadata_snapshot, required_columns); + table_id, row_policy_filter, context, storage, storage_snapshot, metadata_snapshot, required_columns, + prepared_sets); query_info.filter_asts.push_back(row_policy_filter); } @@ -623,7 +625,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (query_info.additional_filter_ast) { additional_filter_info = generateFilterActions( - table_id, query_info.additional_filter_ast, context, storage, storage_snapshot, metadata_snapshot, required_columns); + table_id, query_info.additional_filter_ast, context, storage, storage_snapshot, metadata_snapshot, required_columns, + prepared_sets); additional_filter_info->do_remove_column = true; diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 2e696be4988..1933823f5d2 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -867,3 +867,30 @@ def test_policy_on_distributed_table_via_role(): assert node.query( "SELECT * FROM dist_tbl SETTINGS prefer_localhost_replica=0", user="user1" ) == TSV([[0], [2], [4], [6], [8], [0], [2], [4], [6], [8]]) + + +def test_row_policy_filter_with_subquery(): + copy_policy_xml("no_filters.xml") + assert node.query("SHOW POLICIES") == "" + + node.query("DROP ROW POLICY IF EXISTS filter_1 ON table1") + node.query("DROP TABLE IF EXISTS table_1") + node.query("DROP TABLE IF EXISTS table_2") + + node.query( + "CREATE TABLE table_1 (x int, y int) ENGINE = MergeTree ORDER BY tuple()" + ) + node.query("INSERT INTO table_1 SELECT number, number * number FROM numbers(10)") + + node.query("CREATE TABLE table_2 (a int) ENGINE=MergeTree ORDER BY tuple()") + node.query("INSERT INTO table_2 VALUES (3), (5)") + + node.query( + "CREATE ROW POLICY filter_1 ON table_1 USING x IN (SELECT a FROM table_2) TO ALL" + ) + + assert node.query("SELECT * FROM table_1") == TSV([[3, 9], [5, 25]]) + + node.query("DROP ROW POLICY filter_1 ON table_1") + node.query("DROP TABLE table_1") + node.query("DROP TABLE table_2") From 06d46ca51bfffd95fcab5dcb678c85566d373a1f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 22 Oct 2022 13:28:22 +0000 Subject: [PATCH 043/188] Clear thread::id when ThreadFromGlobalPool exits. --- src/Common/ThreadPool.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 76ada9e0d75..cfeb8deaac5 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -178,7 +178,10 @@ public: func = std::forward(func), args = std::make_tuple(std::forward(args)...)]() mutable /// mutable is needed to destroy capture { - SCOPE_EXIT(state->event.set()); + SCOPE_EXIT( + state->thread_id = std::thread::id(); + state->event.set(); + ); state->thread_id = std::this_thread::get_id(); From f0aed688938281a1782f5acf4803126b6343897b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 Oct 2022 20:53:18 +0200 Subject: [PATCH 044/188] ci/stress: fix memory limits overrides for hung check Signed-off-by: Azat Khuzhin --- docker/test/stress/stress | 2 -- 1 file changed, 2 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 7f3f38bd8f5..a0ec86f7fbe 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -286,9 +286,7 @@ if __name__ == "__main__": # But right now it should work, since neither hung check, nor 00001_select_1 has GROUP BY. "--client-option", "max_untracked_memory=1Gi", - "--client-option", "max_memory_usage_for_user=0", - "--client-option", "memory_profiler_step=1Gi", # Use system database to avoid CREATE/DROP DATABASE queries "--database=system", From b07f65343d0b3171d2dab08d11b90c70a4ee3d93 Mon Sep 17 00:00:00 2001 From: Quanfa Fu Date: Sat, 22 Oct 2022 19:01:33 +0800 Subject: [PATCH 045/188] Add functions: domainRFC, topLevelDomainRFC, domainWithoutWWWRFC... --- .../URL/ExtractFirstSignificantSubdomain.h | 6 +- .../URL/cutToFirstSignificantSubdomain.cpp | 16 ++- .../cutToFirstSignificantSubdomainCustom.cpp | 16 ++- src/Functions/URL/domain.cpp | 5 +- src/Functions/URL/domain.h | 105 +++++++++++++++++- src/Functions/URL/domainWithoutWWW.cpp | 6 +- .../URL/firstSignificantSubdomain.cpp | 5 +- .../URL/firstSignificantSubdomainCustom.cpp | 5 +- src/Functions/URL/port.cpp | 29 ++++- src/Functions/URL/topLevelDomain.cpp | 13 ++- .../0_stateless/00398_url_functions.reference | 18 +++ .../0_stateless/00398_url_functions.sql | 18 +++ ...new_functions_must_be_documented.reference | 10 ++ .../00038_uniq_state_merge2.reference | 4 +- 14 files changed, 227 insertions(+), 29 deletions(-) diff --git a/src/Functions/URL/ExtractFirstSignificantSubdomain.h b/src/Functions/URL/ExtractFirstSignificantSubdomain.h index 73137da474f..0d1b1cac8ef 100644 --- a/src/Functions/URL/ExtractFirstSignificantSubdomain.h +++ b/src/Functions/URL/ExtractFirstSignificantSubdomain.h @@ -16,7 +16,7 @@ struct FirstSignificantSubdomainDefaultLookup } }; -template +template struct ExtractFirstSignificantSubdomain { static size_t getReserveLengthForElement() { return 10; } @@ -35,7 +35,7 @@ struct ExtractFirstSignificantSubdomain Pos tmp; size_t domain_length; - ExtractDomain::execute(data, size, tmp, domain_length); + ExtractDomain::execute(data, size, tmp, domain_length); if (domain_length == 0) return; @@ -105,7 +105,7 @@ struct ExtractFirstSignificantSubdomain Pos tmp; size_t domain_length; - ExtractDomain::execute(data, size, tmp, domain_length); + ExtractDomain::execute(data, size, tmp, domain_length); if (domain_length == 0) return; diff --git a/src/Functions/URL/cutToFirstSignificantSubdomain.cpp b/src/Functions/URL/cutToFirstSignificantSubdomain.cpp index dddfbe4f4dd..10c41b6a4c3 100644 --- a/src/Functions/URL/cutToFirstSignificantSubdomain.cpp +++ b/src/Functions/URL/cutToFirstSignificantSubdomain.cpp @@ -6,7 +6,7 @@ namespace DB { -template +template struct CutToFirstSignificantSubdomain { static size_t getReserveLengthForElement() { return 15; } @@ -19,7 +19,7 @@ struct CutToFirstSignificantSubdomain Pos tmp_data; size_t tmp_length; Pos domain_end; - ExtractFirstSignificantSubdomain::execute(data, size, tmp_data, tmp_length, &domain_end); + ExtractFirstSignificantSubdomain::execute(data, size, tmp_data, tmp_length, &domain_end); if (tmp_length == 0) return; @@ -30,15 +30,23 @@ struct CutToFirstSignificantSubdomain }; struct NameCutToFirstSignificantSubdomain { static constexpr auto name = "cutToFirstSignificantSubdomain"; }; -using FunctionCutToFirstSignificantSubdomain = FunctionStringToString>, NameCutToFirstSignificantSubdomain>; +using FunctionCutToFirstSignificantSubdomain = FunctionStringToString>, NameCutToFirstSignificantSubdomain>; struct NameCutToFirstSignificantSubdomainWithWWW { static constexpr auto name = "cutToFirstSignificantSubdomainWithWWW"; }; -using FunctionCutToFirstSignificantSubdomainWithWWW = FunctionStringToString>, NameCutToFirstSignificantSubdomainWithWWW>; +using FunctionCutToFirstSignificantSubdomainWithWWW = FunctionStringToString>, NameCutToFirstSignificantSubdomainWithWWW>; + +struct NameCutToFirstSignificantSubdomainRFC { static constexpr auto name = "cutToFirstSignificantSubdomainRFC"; }; +using FunctionCutToFirstSignificantSubdomainRFC = FunctionStringToString>, NameCutToFirstSignificantSubdomainRFC>; + +struct NameCutToFirstSignificantSubdomainWithWWWRFC { static constexpr auto name = "cutToFirstSignificantSubdomainWithWWWRFC"; }; +using FunctionCutToFirstSignificantSubdomainWithWWWRFC = FunctionStringToString>, NameCutToFirstSignificantSubdomainWithWWWRFC>; REGISTER_FUNCTION(CutToFirstSignificantSubdomain) { factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/URL/cutToFirstSignificantSubdomainCustom.cpp b/src/Functions/URL/cutToFirstSignificantSubdomainCustom.cpp index a2e51200910..521216c84a7 100644 --- a/src/Functions/URL/cutToFirstSignificantSubdomainCustom.cpp +++ b/src/Functions/URL/cutToFirstSignificantSubdomainCustom.cpp @@ -5,7 +5,7 @@ namespace DB { -template +template struct CutToFirstSignificantSubdomainCustom { static size_t getReserveLengthForElement() { return 15; } @@ -18,7 +18,7 @@ struct CutToFirstSignificantSubdomainCustom Pos tmp_data; size_t tmp_length; Pos domain_end; - ExtractFirstSignificantSubdomain::executeCustom(tld_lookup, data, size, tmp_data, tmp_length, &domain_end); + ExtractFirstSignificantSubdomain::executeCustom(tld_lookup, data, size, tmp_data, tmp_length, &domain_end); if (tmp_length == 0) return; @@ -29,15 +29,23 @@ struct CutToFirstSignificantSubdomainCustom }; struct NameCutToFirstSignificantSubdomainCustom { static constexpr auto name = "cutToFirstSignificantSubdomainCustom"; }; -using FunctionCutToFirstSignificantSubdomainCustom = FunctionCutToFirstSignificantSubdomainCustomImpl, NameCutToFirstSignificantSubdomainCustom>; +using FunctionCutToFirstSignificantSubdomainCustom = FunctionCutToFirstSignificantSubdomainCustomImpl, NameCutToFirstSignificantSubdomainCustom>; struct NameCutToFirstSignificantSubdomainCustomWithWWW { static constexpr auto name = "cutToFirstSignificantSubdomainCustomWithWWW"; }; -using FunctionCutToFirstSignificantSubdomainCustomWithWWW = FunctionCutToFirstSignificantSubdomainCustomImpl, NameCutToFirstSignificantSubdomainCustomWithWWW>; +using FunctionCutToFirstSignificantSubdomainCustomWithWWW = FunctionCutToFirstSignificantSubdomainCustomImpl, NameCutToFirstSignificantSubdomainCustomWithWWW>; + +struct NameCutToFirstSignificantSubdomainCustomRFC { static constexpr auto name = "cutToFirstSignificantSubdomainCustomRFC"; }; +using FunctionCutToFirstSignificantSubdomainCustomRFC = FunctionCutToFirstSignificantSubdomainCustomImpl, NameCutToFirstSignificantSubdomainCustomRFC>; + +struct NameCutToFirstSignificantSubdomainCustomWithWWWRFC { static constexpr auto name = "cutToFirstSignificantSubdomainCustomWithWWWRFC"; }; +using FunctionCutToFirstSignificantSubdomainCustomWithWWWRFC = FunctionCutToFirstSignificantSubdomainCustomImpl, NameCutToFirstSignificantSubdomainCustomWithWWWRFC>; REGISTER_FUNCTION(CutToFirstSignificantSubdomainCustom) { factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/URL/domain.cpp b/src/Functions/URL/domain.cpp index 1d781b37943..e7fead24dc9 100644 --- a/src/Functions/URL/domain.cpp +++ b/src/Functions/URL/domain.cpp @@ -7,12 +7,15 @@ namespace DB { struct NameDomain { static constexpr auto name = "domain"; }; -using FunctionDomain = FunctionStringToString>, NameDomain>; +using FunctionDomain = FunctionStringToString>, NameDomain>; +struct NameDomainRFC { static constexpr auto name = "domainRFC"; }; +using FunctionDomainRFC = FunctionStringToString>, NameDomainRFC>; REGISTER_FUNCTION(Domain) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/URL/domain.h b/src/Functions/URL/domain.h index 92f58fc91e0..64362edf2c3 100644 --- a/src/Functions/URL/domain.h +++ b/src/Functions/URL/domain.h @@ -20,10 +20,10 @@ inline std::string_view checkAndReturnHost(const Pos & pos, const Pos & dot_pos, return std::string_view(start_of_host, pos - start_of_host); } -/// Extracts host from given url. +/// Extracts host from given url (RPC). /// /// @return empty string view if the host is not valid (i.e. it does not have dot, or there no symbol after dot). -inline std::string_view getURLHost(const char * data, size_t size) +inline std::string_view getURLHostRFC(const char * data, size_t size) { Pos pos = data; Pos end = data + size; @@ -129,14 +129,111 @@ done: return checkAndReturnHost(pos, dot_pos, start_of_host); } -template +/// Extracts host from given url. +/// +/// @return empty string view if the host is not valid (i.e. it does not have dot, or there no symbol after dot). +inline std::string_view getURLHost(const char * data, size_t size) +{ + Pos pos = data; + Pos end = data + size; + + if (*pos == '/' && *(pos + 1) == '/') + { + pos += 2; + } + else + { + Pos scheme_end = data + std::min(size, 16UL); + for (++pos; pos < scheme_end; ++pos) + { + if (!isAlphaNumericASCII(*pos)) + { + switch (*pos) + { + case '.': + case '-': + case '+': + break; + case ' ': /// restricted symbols + case '\t': + case '<': + case '>': + case '%': + case '{': + case '}': + case '|': + case '\\': + case '^': + case '~': + case '[': + case ']': + case ';': + case '=': + case '&': + return std::string_view{}; + default: + goto exloop; + } + } + } +exloop: if ((scheme_end - pos) > 2 && *pos == ':' && *(pos + 1) == '/' && *(pos + 2) == '/') + pos += 3; + else + pos = data; + } + + Pos dot_pos = nullptr; + const auto * start_of_host = pos; + for (; pos < end; ++pos) + { + switch (*pos) + { + case '.': + dot_pos = pos; + break; + case ':': /// end symbols + case '/': + case '?': + case '#': + return checkAndReturnHost(pos, dot_pos, start_of_host); + case '@': /// myemail@gmail.com + start_of_host = pos + 1; + break; + case ' ': /// restricted symbols in whole URL + case '\t': + case '<': + case '>': + case '%': + case '{': + case '}': + case '|': + case '\\': + case '^': + case '~': + case '[': + case ']': + case ';': + case '=': + case '&': + return std::string_view{}; + } + } + + return checkAndReturnHost(pos, dot_pos, start_of_host); +} + +template struct ExtractDomain { static size_t getReserveLengthForElement() { return 15; } static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) { - std::string_view host = getURLHost(data, size); + std::string_view host; + if constexpr (conform_rfc) + host = getURLHostRFC(data, size); + else + host = getURLHost(data, size); if (host.empty()) { diff --git a/src/Functions/URL/domainWithoutWWW.cpp b/src/Functions/URL/domainWithoutWWW.cpp index 53ff5bc919e..2fa9159d7af 100644 --- a/src/Functions/URL/domainWithoutWWW.cpp +++ b/src/Functions/URL/domainWithoutWWW.cpp @@ -6,12 +6,16 @@ namespace DB { struct NameDomainWithoutWWW { static constexpr auto name = "domainWithoutWWW"; }; -using FunctionDomainWithoutWWW = FunctionStringToString>, NameDomainWithoutWWW>; +using FunctionDomainWithoutWWW = FunctionStringToString>, NameDomainWithoutWWW>; + +struct NameDomainWithoutWWWRFC { static constexpr auto name = "domainWithoutWWWRFC"; }; +using FunctionDomainWithoutWWWRFC = FunctionStringToString>, NameDomainWithoutWWWRFC>; REGISTER_FUNCTION(DomainWithoutWWW) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/URL/firstSignificantSubdomain.cpp b/src/Functions/URL/firstSignificantSubdomain.cpp index d3aeb90771f..902a4f43fba 100644 --- a/src/Functions/URL/firstSignificantSubdomain.cpp +++ b/src/Functions/URL/firstSignificantSubdomain.cpp @@ -7,12 +7,15 @@ namespace DB { struct NameFirstSignificantSubdomain { static constexpr auto name = "firstSignificantSubdomain"; }; +using FunctionFirstSignificantSubdomain = FunctionStringToString>, NameFirstSignificantSubdomain>; -using FunctionFirstSignificantSubdomain = FunctionStringToString>, NameFirstSignificantSubdomain>; +struct NameFirstSignificantSubdomainRFC { static constexpr auto name = "firstSignificantSubdomainRFC"; }; +using FunctionFirstSignificantSubdomainRFC = FunctionStringToString>, NameFirstSignificantSubdomainRFC>; REGISTER_FUNCTION(FirstSignificantSubdomain) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/URL/firstSignificantSubdomainCustom.cpp b/src/Functions/URL/firstSignificantSubdomainCustom.cpp index f43b42d0309..c07aa2b3ac8 100644 --- a/src/Functions/URL/firstSignificantSubdomainCustom.cpp +++ b/src/Functions/URL/firstSignificantSubdomainCustom.cpp @@ -7,12 +7,15 @@ namespace DB { struct NameFirstSignificantSubdomainCustom { static constexpr auto name = "firstSignificantSubdomainCustom"; }; +using FunctionFirstSignificantSubdomainCustom = FunctionCutToFirstSignificantSubdomainCustomImpl, NameFirstSignificantSubdomainCustom>; -using FunctionFirstSignificantSubdomainCustom = FunctionCutToFirstSignificantSubdomainCustomImpl, NameFirstSignificantSubdomainCustom>; +struct NameFirstSignificantSubdomainCustomRFC { static constexpr auto name = "firstSignificantSubdomainCustomRFC"; }; +using FunctionFirstSignificantSubdomainCustomRFC = FunctionCutToFirstSignificantSubdomainCustomImpl, NameFirstSignificantSubdomainCustomRFC>; REGISTER_FUNCTION(FirstSignificantSubdomainCustom) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/URL/port.cpp b/src/Functions/URL/port.cpp index 85b060ca987..f716f3e454b 100644 --- a/src/Functions/URL/port.cpp +++ b/src/Functions/URL/port.cpp @@ -18,12 +18,9 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -struct FunctionPort : public IFunction +template +struct FunctionPortImpl : public IFunction { - static constexpr auto name = "port"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } bool useDefaultImplementationForConstants() const override { return true; } @@ -94,7 +91,12 @@ private: const char * p = reinterpret_cast(buf.data()) + offset; const char * end = p + size; - std::string_view host = getURLHost(p, size); + std::string_view host; + if constexpr (conform_rfc) + host = getURLHostRFC(p, size); + else + host = getURLHost(p, size); + if (host.empty()) return default_port; if (host.size() == size) @@ -121,9 +123,24 @@ private: } }; +struct FunctionPort : public FunctionPortImpl +{ + static constexpr auto name = "port"; + String getName() const override { return name; } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } +}; + +struct FunctionPortRFC : public FunctionPortImpl +{ + static constexpr auto name = "portRFC"; + String getName() const override { return name; } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } +}; + REGISTER_FUNCTION(Port) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/URL/topLevelDomain.cpp b/src/Functions/URL/topLevelDomain.cpp index 9937618cae9..f5610ed93b7 100644 --- a/src/Functions/URL/topLevelDomain.cpp +++ b/src/Functions/URL/topLevelDomain.cpp @@ -5,13 +5,18 @@ namespace DB { +template struct ExtractTopLevelDomain { static size_t getReserveLengthForElement() { return 5; } static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) { - std::string_view host = getURLHost(data, size); + std::string_view host; + if constexpr (conform_rfc) + host = getURLHostRFC(data, size); + else + host = getURLHost(data, size); res_data = data; res_size = 0; @@ -41,11 +46,15 @@ struct ExtractTopLevelDomain }; struct NameTopLevelDomain { static constexpr auto name = "topLevelDomain"; }; -using FunctionTopLevelDomain = FunctionStringToString, NameTopLevelDomain>; +using FunctionTopLevelDomain = FunctionStringToString>, NameTopLevelDomain>; + +struct NameTopLevelDomainRFC { static constexpr auto name = "topLevelDomainRFC"; }; +using FunctionTopLevelDomainRFC = FunctionStringToString>, NameTopLevelDomainRFC>; REGISTER_FUNCTION(TopLevelDomain) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/00398_url_functions.reference b/tests/queries/0_stateless/00398_url_functions.reference index db3d9224515..2e5a97b380e 100644 --- a/tests/queries/0_stateless/00398_url_functions.reference +++ b/tests/queries/0_stateless/00398_url_functions.reference @@ -7,6 +7,24 @@ http ====HOST==== www.example.com + + + + + + + + + +www.example.com +127.0.0.1 +www.example.com +www.example.com +www.example.com +example.com +example.com +example.com +www.example.com example.com example.com example.com diff --git a/tests/queries/0_stateless/00398_url_functions.sql b/tests/queries/0_stateless/00398_url_functions.sql index 6501b723d23..cbefde7515a 100644 --- a/tests/queries/0_stateless/00398_url_functions.sql +++ b/tests/queries/0_stateless/00398_url_functions.sql @@ -25,6 +25,24 @@ SELECT domain('www.example.com') as Host; SELECT domain('example.com') as Host; SELECT domainWithoutWWW('//paul@www.example.com') AS Host; SELECT domainWithoutWWW('http://paul@www.example.com:80/') AS Host; +SELECT domainRFC('http://paul@www.example.com:80/') AS Host; +SELECT domainRFC('user:password@example.com:8080') AS Host; +SELECT domainRFC('http://user:password@example.com:8080') AS Host; +SELECT domainRFC('http://user:password@example.com:8080/path?query=value#fragment') AS Host; +SELECT domainRFC('newuser:@example.com') AS Host; +SELECT domainRFC('http://:pass@example.com') AS Host; +SELECT domainRFC(':newpass@example.com') AS Host; +SELECT domainRFC('http://user:pass@example@.com') AS Host; +SELECT domainRFC('http://user:pass:example.com') AS Host; +SELECT domainRFC('http:/paul/example/com') AS Host; +SELECT domainRFC('http://www.example.com?q=4') AS Host; +SELECT domainRFC('http://127.0.0.1:443/') AS Host; +SELECT domainRFC('//www.example.com') AS Host; +SELECT domainRFC('//paul@www.example.com') AS Host; +SELECT domainRFC('www.example.com') as Host; +SELECT domainRFC('example.com') as Host; +SELECT domainWithoutWWWRFC('//paul@www.example.com') AS Host; +SELECT domainWithoutWWWRFC('http://paul@www.example.com:80/') AS Host; SELECT '====NETLOC===='; SELECT netloc('http://paul@www.example.com:80/') AS Netloc; diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index c7ac00ee18f..3fd12051f4a 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -221,8 +221,12 @@ cutQueryString cutQueryStringAndFragment cutToFirstSignificantSubdomain cutToFirstSignificantSubdomainCustom +cutToFirstSignificantSubdomainCustomRFC cutToFirstSignificantSubdomainCustomWithWWW +cutToFirstSignificantSubdomainCustomWithWWWRFC +cutToFirstSignificantSubdomainRFC cutToFirstSignificantSubdomainWithWWW +cutToFirstSignificantSubdomainWithWWWRFC cutURLParameter cutWWW dateDiff @@ -281,7 +285,9 @@ dictHas dictIsIn divide domain +domainRFC domainWithoutWWW +domainWithoutWWWRFC dotProduct dumpColumnStructure e @@ -332,6 +338,8 @@ filesystemFree finalizeAggregation firstSignificantSubdomain firstSignificantSubdomainCustom +firstSignificantSubdomainCustomRFC +firstSignificantSubdomainRFC flattenTuple floor format @@ -593,6 +601,7 @@ polygonsUnionSpherical polygonsWithinCartesian polygonsWithinSpherical port +portRFC position positionCaseInsensitive positionCaseInsensitiveUTF8 @@ -898,6 +907,7 @@ toYearWeek today tokens topLevelDomain +topLevelDomainRFC transactionID transactionLatestSnapshot transactionOldestSnapshot diff --git a/tests/queries/1_stateful/00038_uniq_state_merge2.reference b/tests/queries/1_stateful/00038_uniq_state_merge2.reference index fea0619e225..0b335faaa67 100644 --- a/tests/queries/1_stateful/00038_uniq_state_merge2.reference +++ b/tests/queries/1_stateful/00038_uniq_state_merge2.reference @@ -1,5 +1,5 @@ -ru 262915 69218 - 91871 89417 +ru 262914 69218 + 91872 89417 com 63298 30285 ua 29037 17475 html 25079 15039 From 30f4f33e7929c7241d260aed526dca96936372fb Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 24 Oct 2022 01:39:40 +0000 Subject: [PATCH 046/188] Add test --- .../0_stateless/02469_fix_aliases_parser.reference | 2 ++ tests/queries/0_stateless/02469_fix_aliases_parser.sql | 9 +++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/02469_fix_aliases_parser.reference create mode 100644 tests/queries/0_stateless/02469_fix_aliases_parser.sql diff --git a/tests/queries/0_stateless/02469_fix_aliases_parser.reference b/tests/queries/0_stateless/02469_fix_aliases_parser.reference new file mode 100644 index 00000000000..09f584c9cd4 --- /dev/null +++ b/tests/queries/0_stateless/02469_fix_aliases_parser.reference @@ -0,0 +1,2 @@ +45 +[0] diff --git a/tests/queries/0_stateless/02469_fix_aliases_parser.sql b/tests/queries/0_stateless/02469_fix_aliases_parser.sql new file mode 100644 index 00000000000..227d8becdb6 --- /dev/null +++ b/tests/queries/0_stateless/02469_fix_aliases_parser.sql @@ -0,0 +1,9 @@ +SELECT sum(number number number) FROM numbers(10); -- { clientError 62 } +SELECT sum(number number) FROM numbers(10); -- { clientError 62 } +SELECT sum(number AS number) FROM numbers(10); + +SELECT [number number number] FROM numbers(1); -- { clientError 62 } +SELECT [number number] FROM numbers(1); -- { clientError 62 } +SELECT [number AS number] FROM numbers(1); + +SELECT cast('1234' lhs lhs, 'UInt32'), lhs; -- { clientError 62 } \ No newline at end of file From 27494aea6366655881f198d06ef9246f1755d7d1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 16 Oct 2022 23:31:03 +0200 Subject: [PATCH 047/188] Make getResource() independent from the order of the sections It is possible for end section goes before begin section for some resource, and this case it will not find it. This is what happens here [1]: [1]: https://s3.amazonaws.com/clickhouse-test-reports/41988/053fec3f451e97ac41b6c223d95013b758a9a330/fast_test.html Signed-off-by: Azat Khuzhin --- src/Common/SymbolIndex.cpp | 24 +++++++++++++----------- src/Common/SymbolIndex.h | 13 ++++++++++--- 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index e217d23cc27..6f31009b1d2 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -99,23 +99,25 @@ void updateResources(ElfW(Addr) base_address, std::string_view object_name, std: name = name.substr((name[0] == '_') + strlen("binary_")); name = name.substr(0, name.size() - strlen("_start")); - resources.emplace(name, SymbolIndex::ResourcesBlob{ - base_address, - object_name, - std::string_view{char_address, 0}, // NOLINT - }); + auto & resource = resources[name]; + if (!resource.base_address || resource.base_address == base_address) + { + resource.base_address = base_address; + resource.start = std::string_view{char_address, 0}; // NOLINT(bugprone-string-constructor) + resource.object_name = object_name; + } } - else if (name.ends_with("_end")) + if (name.ends_with("_end")) { name = name.substr((name[0] == '_') + strlen("binary_")); name = name.substr(0, name.size() - strlen("_end")); - auto it = resources.find(name); - if (it != resources.end() && it->second.base_address == base_address && it->second.data.empty()) + auto & resource = resources[name]; + if (!resource.base_address || resource.base_address == base_address) { - const char * start = it->second.data.data(); - assert(char_address >= start); - it->second.data = std::string_view{start, static_cast(char_address - start)}; + resource.base_address = base_address; + resource.end = std::string_view{char_address, 0}; // NOLINT(bugprone-string-constructor) + resource.object_name = object_name; } } } diff --git a/src/Common/SymbolIndex.h b/src/Common/SymbolIndex.h index f2b40f02ead..47162331946 100644 --- a/src/Common/SymbolIndex.h +++ b/src/Common/SymbolIndex.h @@ -51,7 +51,7 @@ public: std::string_view getResource(String name) const { if (auto it = data.resources.find(name); it != data.resources.end()) - return it->second.data; + return it->second.data(); return {}; } @@ -63,11 +63,18 @@ public: { /// Symbol can be presented in multiple shared objects, /// base_address will be used to compare only symbols from the same SO. - ElfW(Addr) base_address; + ElfW(Addr) base_address = 0; /// Just a human name of the SO. std::string_view object_name; /// Data blob. - std::string_view data; + std::string_view start; + std::string_view end; + + std::string_view data() const + { + assert(end.data() >= start.data()); + return std::string_view{start.data(), static_cast(end.data() - start.data())}; + } }; using Resources = std::unordered_map; From 4ddbdead4a57b28d89e30db5bd3928ab7fa71d76 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 10:17:56 +0200 Subject: [PATCH 048/188] Add changelog for 22.10 --- CHANGELOG.md | 129 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 129 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 83c1cbf1eb4..4c7ca701d53 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v22.10, 2022-10-25](#2210)**
**[ClickHouse release v22.9, 2022-09-22](#229)**
**[ClickHouse release v22.8-lts, 2022-08-18](#228)**
**[ClickHouse release v22.7, 2022-07-21](#227)**
@@ -10,6 +11,134 @@ **[ClickHouse release v22.1, 2022-01-18](#221)**
**[Changelog for 2021](https://clickhouse.com/docs/en/whats-new/changelog/2021/)**
+### ClickHouse release 22.10, 2022-10-25 + +#### Backward Incompatible Change +* Rename cache commands: `show caches` -> `show filesystem caches`, `describe cache` -> `describe filesystem cache`. [#41508](https://github.com/ClickHouse/ClickHouse/pull/41508) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove support for the `WITH TIMEOUT` section for `LIVE VIEW`. This closes [#40557](https://github.com/ClickHouse/ClickHouse/issues/40557). [#42173](https://github.com/ClickHouse/ClickHouse/pull/42173) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove support for the `{database}` macro from the client's prompt. It was displayed incorrectly if the database was unspecified and it was not updated on `USE` statements. This closes [#25891](https://github.com/ClickHouse/ClickHouse/issues/25891). [#42508](https://github.com/ClickHouse/ClickHouse/pull/42508) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Composable protocol configuration is added. Now different protocols can be set up with different listen hosts. Protocol wrappers such as PROXYv1 can be set up over any other protocols (TCP, TCP secure, MySQL, Postgres). [#41198](https://github.com/ClickHouse/ClickHouse/pull/41198) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Add `S3` as a new type of the destination of backups. Support BACKUP to S3 with as-is path/data structure. [#42333](https://github.com/ClickHouse/ClickHouse/pull/42333) ([Vitaly Baranov](https://github.com/vitlibar)), [#42232](https://github.com/ClickHouse/ClickHouse/pull/42232) ([Azat Khuzhin](https://github.com/azat)). +* Added functions (`randUniform`, `randNormal`, `randLogNormal`, `randExponential`, `randChiSquared`, `randStudentT`, `randFisherF`, `randBernoulli`, `randBinomial`, `randNegativeBinomial`, `randPoisson` ) to generate random values according to the specified distributions. This closes [#21834](https://github.com/ClickHouse/ClickHouse/issues/21834). [#42411](https://github.com/ClickHouse/ClickHouse/pull/42411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* An improvement for ClickHouse Keeper: add support for uploading snapshots to S3. S3 information can be defined inside `keeper_server.s3_snapshot`. [#41342](https://github.com/ClickHouse/ClickHouse/pull/41342) ([Antonio Andelic](https://github.com/antonio2368)). +* Added an aggregate function `analysisOfVariance` (`anova`) to perform a statistical test over several groups of normally distributed observations to find out whether all groups have the same mean or not. Original PR [#37872](https://github.com/ClickHouse/ClickHouse/issues/37872). [#42131](https://github.com/ClickHouse/ClickHouse/pull/42131) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Support limiting of temporary data stored on disk using settings `max_temporary_data_on_disk_size_for_user`/`max_temporary_data_on_disk_size_for_query` . [#40893](https://github.com/ClickHouse/ClickHouse/pull/40893) ([Vladimir C](https://github.com/vdimir)). +* Add setting `format_json_object_each_row_column_for_object_name` to write/parse object name as column value in JSONObjectEachRow format. [#41703](https://github.com/ClickHouse/ClickHouse/pull/41703) ([Kruglov Pavel](https://github.com/Avogar)). +* Add BLAKE3 hash-function to SQL. [#33435](https://github.com/ClickHouse/ClickHouse/pull/33435) ([BoloniniD](https://github.com/BoloniniD)). +* The function `javaHash` has been extended to integers. [#41131](https://github.com/ClickHouse/ClickHouse/pull/41131) ([JackyWoo](https://github.com/JackyWoo)). +* Add OpenTelemetry support to ON CLUSTER DDL (require `distributed_ddl_entry_format_version` to be set to 4). [#41484](https://github.com/ClickHouse/ClickHouse/pull/41484) ([Frank Chen](https://github.com/FrankChen021)). +* Added system table `asynchronous_insert_log`. It contains information about asynchronous inserts (including results of queries in fire-and-forget mode (with `wait_for_async_insert=0`)) for better introspection. [#42040](https://github.com/ClickHouse/ClickHouse/pull/42040) ([Anton Popov](https://github.com/CurtizJ)). +* Add support for methods `lz4`, `bz2`, `snappy` in HTTP's `Accept-Encoding` which is a non-standard extension to HTTP protocol. [#42071](https://github.com/ClickHouse/ClickHouse/pull/42071) ([Nikolay Degterinsky](https://github.com/evillique)). + +#### Experimental Feature +* Initial implementation of Kusto Query Language. Please don't use it. [#37961](https://github.com/ClickHouse/ClickHouse/pull/37961) ([Yong Wang](https://github.com/kashwy)). + +#### Performance Improvement +* Relax the "Too many parts" threshold. This closes [#6551](https://github.com/ClickHouse/ClickHouse/issues/6551). Now ClickHouse will allow more parts in a partition if the average part size is large enough (at least 10 GiB). This allows to have up to petabytes of data in a single partition of a single table on a single server, which is possible using disk shelves or object storage. [#42002](https://github.com/ClickHouse/ClickHouse/pull/42002) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Implement operator precedence element parser to resolve stack overflow issues and make the required stack size smaller. [#34892](https://github.com/ClickHouse/ClickHouse/pull/34892) ([Nikolay Degterinsky](https://github.com/evillique)). +* DISTINCT in order optimization leverage sorting properties of data streams. This improvement will enable reading in order for DISTINCT if applicable (before it was necessary to provide ORDER BY for columns in DISTINCT). [#41014](https://github.com/ClickHouse/ClickHouse/pull/41014) ([Igor Nikonov](https://github.com/devcrafter)). +* ColumnVector: optimize UInt8 index with AVX512VBMI. [#41247](https://github.com/ClickHouse/ClickHouse/pull/41247) ([Guo Wangyang](https://github.com/guowangy)). +* Optimize the lock contentions for `ThreadGroupStatus::mutex`. The performance experiments of **SSB** (Star Schema Benchmark) on the ICX device (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads) shows that this change could bring a **2.95x** improvement of the geomean of all subcases' QPS. [#41675](https://github.com/ClickHouse/ClickHouse/pull/41675) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Add `ldapr` capabilities to AArch64 builds. This is supported from Graviton 2+, Azure and GCP instances. Only appeared in clang-15 [not so long ago](https://github.com/llvm/llvm-project/commit/9609b5daffe9fd28d83d83da895abc5113f76c24). [#41778](https://github.com/ClickHouse/ClickHouse/pull/41778) ([Daniel Kutenin](https://github.com/danlark1)). +* Improve performance when comparing strings and one argument is empty constant string. [#41870](https://github.com/ClickHouse/ClickHouse/pull/41870) ([Jiebin Sun](https://github.com/jiebinn)). +* Optimize `insertFrom` of ColumnAggregateFunction to share Aggregate State in some cases. [#41960](https://github.com/ClickHouse/ClickHouse/pull/41960) ([flynn](https://github.com/ucasfl)). +* Make writing to `azure_blob_storage` disks faster (respect `max_single_part_upload_size` instead of writing a block per each buffer size). Inefficiency mentioned in [#41754](https://github.com/ClickHouse/ClickHouse/issues/41754). [#42041](https://github.com/ClickHouse/ClickHouse/pull/42041) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Make thread ids in the process list and query_log unique to avoid waste. [#42180](https://github.com/ClickHouse/ClickHouse/pull/42180) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support skipping cache completely (both download to cache and reading cached data) in case the requested read range exceeds the threshold defined by cache setting `bypass_cache_threashold`, requires to be enabled with `enable_bypass_cache_with_threshold`). [#42418](https://github.com/ClickHouse/ClickHouse/pull/42418) ([Han Shukai](https://github.com/KinderRiven)). This helps on slow local disks. + +#### Improvement +* Add setting `allow_implicit_no_password`: with combination with `allow_no_password` it forbids creating a user with no password unless `IDENTIFIED WITH no_password` is explicitly specified. [#41341](https://github.com/ClickHouse/ClickHouse/pull/41341) ([Nikolay Degterinsky](https://github.com/evillique)). +* Embedded Keeper will always start in the background allowing ClickHouse to start without achieving quorum. [#40991](https://github.com/ClickHouse/ClickHouse/pull/40991) ([Antonio Andelic](https://github.com/antonio2368)). +* Made reestablishing a new connection to ZooKeeper more reactive in case of expiration of the previous one. Previously there was a task which spawns every minute by default and thus a table could be in readonly state for about this time. [#41092](https://github.com/ClickHouse/ClickHouse/pull/41092) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Now projections can be used with zero copy replication (zero-copy replication is a non-production feature). [#41147](https://github.com/ClickHouse/ClickHouse/pull/41147) ([alesapin](https://github.com/alesapin)). +* Support expression `(EXPLAIN SELECT ...)` in a subquery. Queries like `SELECT * FROM (EXPLAIN PIPELINE SELECT col FROM TABLE ORDER BY col)` became valid. [#40630](https://github.com/ClickHouse/ClickHouse/pull/40630) ([Vladimir C](https://github.com/vdimir)). +* Allow changing `async_insert_max_data_size` or `async_insert_busy_timeout_ms` in scope of query. E.g. user wants to insert data rarely and she doesn't have an access to server config to tune default settings. [#40668](https://github.com/ClickHouse/ClickHouse/pull/40668) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Improvements for reading from remote filesystems, made threadpool size for reads/writes configurable. Closes [#41070](https://github.com/ClickHouse/ClickHouse/issues/41070). [#41011](https://github.com/ClickHouse/ClickHouse/pull/41011) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support all combinators combination in WindowTransform/arratReduce*/initializeAggregation/aggregate functions versioning. Previously combinators like `ForEach/Resample/Map` didn't work in these places, using them led to exception like`State function ... inserts results into non-state column`. [#41107](https://github.com/ClickHouse/ClickHouse/pull/41107) ([Kruglov Pavel](https://github.com/Avogar)). +* Add function `tryDecrypt` that returns NULL when decrypt fail (e.g. decrypt with incorrect key) instead of throwing exception. [#41206](https://github.com/ClickHouse/ClickHouse/pull/41206) ([Duc Canh Le](https://github.com/canhld94)). +* Add the `unreserved_space` column to the `system.disks` table to check how much space is not taken by reservations per disk. [#41254](https://github.com/ClickHouse/ClickHouse/pull/41254) ([filimonov](https://github.com/filimonov)). +* Support s3 authorization headers in table function arguments. [#41261](https://github.com/ClickHouse/ClickHouse/pull/41261) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add support for MultiRead in Keeper and internal ZooKeeper client (this is an extension to ZooKeeper protocol, only available in ClickHouse Keeper). [#41410](https://github.com/ClickHouse/ClickHouse/pull/41410) ([Antonio Andelic](https://github.com/antonio2368)). +* Add support for decimal type comparing with floating point literal in IN operator. [#41544](https://github.com/ClickHouse/ClickHouse/pull/41544) ([liang.huang](https://github.com/lhuang09287750)). +* Allow readable size values (like `1TB`) in cache config. [#41688](https://github.com/ClickHouse/ClickHouse/pull/41688) ([Kseniia Sumarokova](https://github.com/kssenii)). +* ClickHouse could cache stale DNS entries for some period of time (15 seconds by default) until the cache won't be updated asynchronously. During these period ClickHouse can nevertheless try to establish a connection and produce errors. This behaviour is fixed. [#41707](https://github.com/ClickHouse/ClickHouse/pull/41707) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add interactive history search with fzf-like utility (fzf/sk) for `clickhouse-client`/`clickhouse-local` (note you can use `FZF_DEFAULT_OPTS`/`SKIM_DEFAULT_OPTIONS` to additionally configure the behavior). [#41730](https://github.com/ClickHouse/ClickHouse/pull/41730) ([Azat Khuzhin](https://github.com/azat)). +* For client when connecting to a secure server with invalid certificate only allow to proceed with '--accept-certificate' flag. [#41743](https://github.com/ClickHouse/ClickHouse/pull/41743) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Add function `tryBase58Decode`, similar to the existing function `tryBase64Decode`. [#41824](https://github.com/ClickHouse/ClickHouse/pull/41824) ([Robert Schulze](https://github.com/rschu1ze)). +* Improve feedback when replacing partition with different primary key. Fixes [#34798](https://github.com/ClickHouse/ClickHouse/issues/34798). [#41838](https://github.com/ClickHouse/ClickHouse/pull/41838) ([Salvatore](https://github.com/tbsal)). +* Fix parallel parsing: segmentator now checks `max_block_size`. This fixed memory overallocation in case of parallel parsing and small LIMIT. [#41852](https://github.com/ClickHouse/ClickHouse/pull/41852) ([Vitaly Baranov](https://github.com/vitlibar)). +* Don't add "TABLE_IS_DROPPED" exception to `system.errors` if it's happened during SELECT from a system table and was ignored. [#41908](https://github.com/ClickHouse/ClickHouse/pull/41908) ([AlfVII](https://github.com/AlfVII)). +* Improve option `enable_extended_results_for_datetime_functions` to return results of type DateTime64 for functions `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute` and `timeSlot`. [#41910](https://github.com/ClickHouse/ClickHouse/pull/41910) ([Roman Vasin](https://github.com/rvasin)). +* Improve `DateTime` type inference for text formats. Now it respect setting `date_time_input_format` and doesn't try to infer datetimes from numbers as timestamps. Closes [#41389](https://github.com/ClickHouse/ClickHouse/issues/41389) Closes [#42206](https://github.com/ClickHouse/ClickHouse/issues/42206). [#41912](https://github.com/ClickHouse/ClickHouse/pull/41912) ([Kruglov Pavel](https://github.com/Avogar)). +* Remove confusing warning when inserting with `perform_ttl_move_on_insert` = false. [#41980](https://github.com/ClickHouse/ClickHouse/pull/41980) ([Vitaly Baranov](https://github.com/vitlibar)). +* Allow user to write `countState(*)` similar to `count(*)`. This closes [#9338](https://github.com/ClickHouse/ClickHouse/issues/9338). [#41983](https://github.com/ClickHouse/ClickHouse/pull/41983) ([Amos Bird](https://github.com/amosbird)). +* Fix `rankCorr` size overflow. [#42020](https://github.com/ClickHouse/ClickHouse/pull/42020) ([Duc Canh Le](https://github.com/canhld94)). +* Added an option to specify an arbitrary string as an environment name in the Sentry's config for more handy reports. [#42037](https://github.com/ClickHouse/ClickHouse/pull/42037) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix parsing out-of-range Date from CSV. [#42044](https://github.com/ClickHouse/ClickHouse/pull/42044) ([Andrey Zvonov](https://github.com/zvonand)). +* `parseDataTimeBestEffort` now supports comma between date and time. Closes [#42038](https://github.com/ClickHouse/ClickHouse/issues/42038). [#42049](https://github.com/ClickHouse/ClickHouse/pull/42049) ([flynn](https://github.com/ucasfl)). +* Improved stale replica recovery process for `ReplicatedMergeTree`. If a lost replica have some parts which absent on a healthy replica, but these parts should appear in future according to replication queue of the healthy replica, then lost replica will keep such parts instead of detaching them. [#42134](https://github.com/ClickHouse/ClickHouse/pull/42134) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add a possibility to use `Date32` arguments for date_diff function. Fix issue in date_diff function when using DateTime64 arguments with start date before Unix epoch and end date after Unix epoch. [#42308](https://github.com/ClickHouse/ClickHouse/pull/42308) ([Roman Vasin](https://github.com/rvasin)). +* When uploading big parts to Minio, 'Complete Multipart Upload' can take a long time. Minio sends heartbeats every 10 seconds (see https://github.com/minio/minio/pull/7198). But clickhouse times out earlier, because the default send/receive timeout is [set](https://github.com/ClickHouse/ClickHouse/blob/cc24fcd6d5dfb67f5f66f5483e986bd1010ad9cf/src/IO/S3/PocoHTTPClient.cpp#L123) to 5 seconds. [#42321](https://github.com/ClickHouse/ClickHouse/pull/42321) ([filimonov](https://github.com/filimonov)). +* Fix rarely invalid cast of aggregate state types with complex types such as Decimal. This fixes [#42408](https://github.com/ClickHouse/ClickHouse/issues/42408). [#42417](https://github.com/ClickHouse/ClickHouse/pull/42417) ([Amos Bird](https://github.com/amosbird)). +* Allow to use `Date32` arguments for `dateName` function. [#42554](https://github.com/ClickHouse/ClickHouse/pull/42554) ([Roman Vasin](https://github.com/rvasin)). +* Now filters with NULL literals will be used during index analysis. [#34063](https://github.com/ClickHouse/ClickHouse/issues/34063). [#41842](https://github.com/ClickHouse/ClickHouse/pull/41842) ([Amos Bird](https://github.com/amosbird)). + +#### Build/Testing/Packaging Improvement +* Add fuzzer for table definitions [#40096](https://github.com/ClickHouse/ClickHouse/pull/40096) ([Anton Popov](https://github.com/CurtizJ)). This represents the biggest advancement for ClickHouse testing in this year so far. +* Beta version of the ClickHouse Cloud service is released: [https://clickhouse.cloud/](https://clickhouse.cloud/). It provides the easiest way to use ClickHouse (even slightly easier than the single-command installation). +* Added support of WHERE clause generation to AST Fuzzer and possibility to add or remove ORDER BY and WHERE clause. [#38519](https://github.com/ClickHouse/ClickHouse/pull/38519) ([Ilya Yatsishin](https://github.com/qoega)). +* Aarch64 binaries now require at least ARMv8.2, released in 2016. Most notably, this enables use of ARM LSE, i.e. native atomic operations. Also, CMake build option "NO_ARMV81_OR_HIGHER" has been added to allow compilation of binaries for older ARMv8.0 hardware, e.g. Raspberry Pi 4. [#41610](https://github.com/ClickHouse/ClickHouse/pull/41610) ([Robert Schulze](https://github.com/rschu1ze)). +* Allow building ClickHouse with Musl (small changes after it was already supported but broken). [#41987](https://github.com/ClickHouse/ClickHouse/pull/41987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add the `$CLICKHOUSE_CRONFILE` file checking to avoid running the `sed` command to get the file not found error on install. [#42081](https://github.com/ClickHouse/ClickHouse/pull/42081) ([Chun-Sheng, Li](https://github.com/peter279k)). +* Update cctz to `2022e` to support the new timezone changes. Palestine transitions are now Saturdays at 02:00. Simplify three Ukraine zones into one. Jordan and Syria switch from +02/+03 with DST to year-round +03. (https://data.iana.org/time-zones/tzdb/NEWS). This closes [#42252](https://github.com/ClickHouse/ClickHouse/issues/42252). [#42327](https://github.com/ClickHouse/ClickHouse/pull/42327) ([Alexey Milovidov](https://github.com/alexey-milovidov)). [#42273](https://github.com/ClickHouse/ClickHouse/pull/42273) ([Dom Del Nano](https://github.com/ddelnano)). +* Add Rust code support into ClickHouse with BLAKE3 hash-function library as an example. [#33435](https://github.com/ClickHouse/ClickHouse/pull/33435) ([BoloniniD](https://github.com/BoloniniD)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Choose correct aggregation method for `LowCardinality` with big integer types. [#42342](https://github.com/ClickHouse/ClickHouse/pull/42342) ([Duc Canh Le](https://github.com/canhld94)). +* Several fixes for `web` disk. [#41652](https://github.com/ClickHouse/ClickHouse/pull/41652) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixes issue when docker run will fail if `https_port` is not present in config. [#41693](https://github.com/ClickHouse/ClickHouse/pull/41693) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Mutations were not cancelled properly on server shutdown or `SYSTEM STOP MERGES` query and cancellation might take long time, it's fixed. [#41699](https://github.com/ClickHouse/ClickHouse/pull/41699) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix wrong result of queries with `ORDER BY` or `GROUP BY` by columns from prefix of sorting key, wrapped into monotonic functions, with enable "read in order" optimization (settings `optimize_read_in_order` and `optimize_aggregation_in_order`). [#41701](https://github.com/ClickHouse/ClickHouse/pull/41701) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible crash in `SELECT` from `Merge` table with enabled `optimize_monotonous_functions_in_order_by` setting. Fixes [#41269](https://github.com/ClickHouse/ClickHouse/issues/41269). [#41740](https://github.com/ClickHouse/ClickHouse/pull/41740) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed "Part ... intersects part ..." error that might happen in extremely rare cases if replica was restarted just after detaching some part as broken. [#41741](https://github.com/ClickHouse/ClickHouse/pull/41741) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Don't allow to create or alter merge tree tables with column name `_row_exists`, which is reserved for lightweight delete. Fixed [#41716](https://github.com/ClickHouse/ClickHouse/issues/41716). [#41763](https://github.com/ClickHouse/ClickHouse/pull/41763) ([Jianmei Zhang](https://github.com/zhangjmruc)). +* Fix a bug that CORS headers are missing in some HTTP responses. [#41792](https://github.com/ClickHouse/ClickHouse/pull/41792) ([Frank Chen](https://github.com/FrankChen021)). +* 22.9 might fail to startup `ReplicatedMergeTree` table if that table was created by 20.3 or older version and was never altered, it's fixed. Fixes [#41742](https://github.com/ClickHouse/ClickHouse/issues/41742). [#41796](https://github.com/ClickHouse/ClickHouse/pull/41796) ([Alexander Tokmakov](https://github.com/tavplubix)). +* When the batch sending fails for some reason, it cannot be automatically recovered, and if it is not processed in time, it will lead to accumulation, and the printed error message will become longer and longer, which will cause the http thread to block. [#41813](https://github.com/ClickHouse/ClickHouse/pull/41813) ([zhongyuankai](https://github.com/zhongyuankai)). +* Fix compact parts with compressed marks setting. Fixes [#41783](https://github.com/ClickHouse/ClickHouse/issues/41783) and [#41746](https://github.com/ClickHouse/ClickHouse/issues/41746). [#41823](https://github.com/ClickHouse/ClickHouse/pull/41823) ([alesapin](https://github.com/alesapin)). +* Old versions of Replicated database doesn't have a special marker in [Zoo]Keeper. We need to check only whether the node contains come obscure data instead of special mark. [#41875](https://github.com/ClickHouse/ClickHouse/pull/41875) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix possible exception in fs cache. [#41884](https://github.com/ClickHouse/ClickHouse/pull/41884) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix `use_environment_credentials` for s3 table function. [#41970](https://github.com/ClickHouse/ClickHouse/pull/41970) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed "Directory already exists and is not empty" error on detaching broken part that might prevent `ReplicatedMergeTree` table from starting replication. Fixes [#40957](https://github.com/ClickHouse/ClickHouse/issues/40957). [#41981](https://github.com/ClickHouse/ClickHouse/pull/41981) ([Alexander Tokmakov](https://github.com/tavplubix)). +* `toDateTime64` now returns the same output with negative integer and float arguments. [#42025](https://github.com/ClickHouse/ClickHouse/pull/42025) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix write into `azure_blob_storage`. Partially closes [#41754](https://github.com/ClickHouse/ClickHouse/issues/41754). [#42034](https://github.com/ClickHouse/ClickHouse/pull/42034) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix the `bzip2` decoding issue for specific `bzip2` files. [#42046](https://github.com/ClickHouse/ClickHouse/pull/42046) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix SQL function `toLastDayOfMonth` with setting "enable_extended_results_for_datetime_functions = 1" at the beginning of the extended range (January 1900). - Fix SQL function "toRelativeWeekNum()" with setting "enable_extended_results_for_datetime_functions = 1" at the end of extended range (December 2299). - Improve the performance of for SQL functions "toISOYear()", "toFirstDayNumOfISOYearIndex()" and "toYearWeekOfNewyearMode()" by avoiding unnecessary index arithmetics. [#42084](https://github.com/ClickHouse/ClickHouse/pull/42084) ([Roman Vasin](https://github.com/rvasin)). +* The maximum size of fetches for each table accidentally was set to 8 while the pool size could be bigger. Now the maximum size of fetches for table is equal to the pool size. [#42090](https://github.com/ClickHouse/ClickHouse/pull/42090) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* A table might be shut down and a dictionary might be detached before checking if can be dropped without breaking dependencies between table, it's fixed. Fixes [#41982](https://github.com/ClickHouse/ClickHouse/issues/41982). [#42106](https://github.com/ClickHouse/ClickHouse/pull/42106) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix bad inefficiency of `remote_filesystem_read_method=read` with filesystem cache. Closes [#42125](https://github.com/ClickHouse/ClickHouse/issues/42125). [#42129](https://github.com/ClickHouse/ClickHouse/pull/42129) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix possible timeout exception for distributed queries with use_hedged_requests = 0. [#42130](https://github.com/ClickHouse/ClickHouse/pull/42130) ([Azat Khuzhin](https://github.com/azat)). +* Fixed a minor bug inside function `runningDifference` in case of using it with `Date32` type. Previously `Date` was used and it may cause some logical errors like `Bad cast from type DB::ColumnVector to DB::ColumnVector'`. [#42143](https://github.com/ClickHouse/ClickHouse/pull/42143) ([Alfred Xu](https://github.com/sperlingxx)). +* Fix reusing of files > 4GB from base backup. [#42146](https://github.com/ClickHouse/ClickHouse/pull/42146) ([Azat Khuzhin](https://github.com/azat)). +* DISTINCT in order fails with LOGICAL_ERROR if first column in sorting key contains function. [#42186](https://github.com/ClickHouse/ClickHouse/pull/42186) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix a bug with projections and the `aggregate_functions_null_for_empty` setting. This bug is very rare and appears only if you enable the `aggregate_functions_null_for_empty` setting in the server's config. This closes [#41647](https://github.com/ClickHouse/ClickHouse/issues/41647). [#42198](https://github.com/ClickHouse/ClickHouse/pull/42198) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix read from `Buffer` tables with read in order desc. [#42236](https://github.com/ClickHouse/ClickHouse/pull/42236) ([Duc Canh Le](https://github.com/canhld94)). +* Fix a bug which prevents ClickHouse to start when `background_pool_size setting` is set on default profile but `background_merges_mutations_concurrency_ratio` is not. [#42315](https://github.com/ClickHouse/ClickHouse/pull/42315) ([nvartolomei](https://github.com/nvartolomei)). +* `ALTER UPDATE` of attached part (with columns different from table schema) could create an invalid `columns.txt` metadata on disk. Reading from such part could fail with errors or return invalid data. Fixes [#42161](https://github.com/ClickHouse/ClickHouse/issues/42161). [#42319](https://github.com/ClickHouse/ClickHouse/pull/42319) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Setting `additional_table_filters` were not applied to `Distributed` storage. Fixes [#41692](https://github.com/ClickHouse/ClickHouse/issues/41692). [#42322](https://github.com/ClickHouse/ClickHouse/pull/42322) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix a data race in query finish/cancel. This closes [#42346](https://github.com/ClickHouse/ClickHouse/issues/42346). [#42362](https://github.com/ClickHouse/ClickHouse/pull/42362) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* This reverts [#40217](https://github.com/ClickHouse/ClickHouse/issues/40217) which introduced a regression in date/time functions. [#42367](https://github.com/ClickHouse/ClickHouse/pull/42367) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix assert cast in join on falsy condition, Close [#42380](https://github.com/ClickHouse/ClickHouse/issues/42380). [#42407](https://github.com/ClickHouse/ClickHouse/pull/42407) ([Vladimir C](https://github.com/vdimir)). +* Fix buffer overflow in the processing of Decimal data types. This closes [#42451](https://github.com/ClickHouse/ClickHouse/issues/42451). [#42465](https://github.com/ClickHouse/ClickHouse/pull/42465) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* `AggregateFunctionQuantile` now correctly works with UInt128 columns. Previously, the quantile state interpreted `UInt128` columns as `Int128` which could have led to incorrect results. [#42473](https://github.com/ClickHouse/ClickHouse/pull/42473) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix bad_cast assert during INSERT into `Annoy` indexes over non-Float32 columns. `Annoy` indices is an experimental feature. [#42485](https://github.com/ClickHouse/ClickHouse/pull/42485) ([Robert Schulze](https://github.com/rschu1ze)). +* Arithmetic operator with Date or DateTime and 128 or 256-bit integer was referencing uninitialized memory. [#42453](https://github.com/ClickHouse/ClickHouse/issues/42453). [#42573](https://github.com/ClickHouse/ClickHouse/pull/42573) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix unexpected table loading error when partition key contains alias function names during server upgrade. [#36379](https://github.com/ClickHouse/ClickHouse/pull/36379) ([Amos Bird](https://github.com/amosbird)). + + ### ClickHouse release 22.9, 2022-09-22 #### Backward Incompatible Change From b15850984f0aada23c9281f45b33488665995d86 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 11:22:07 +0300 Subject: [PATCH 049/188] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4c7ca701d53..b02d3d9ac84 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,7 +21,7 @@ #### New Feature * Composable protocol configuration is added. Now different protocols can be set up with different listen hosts. Protocol wrappers such as PROXYv1 can be set up over any other protocols (TCP, TCP secure, MySQL, Postgres). [#41198](https://github.com/ClickHouse/ClickHouse/pull/41198) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). * Add `S3` as a new type of the destination of backups. Support BACKUP to S3 with as-is path/data structure. [#42333](https://github.com/ClickHouse/ClickHouse/pull/42333) ([Vitaly Baranov](https://github.com/vitlibar)), [#42232](https://github.com/ClickHouse/ClickHouse/pull/42232) ([Azat Khuzhin](https://github.com/azat)). -* Added functions (`randUniform`, `randNormal`, `randLogNormal`, `randExponential`, `randChiSquared`, `randStudentT`, `randFisherF`, `randBernoulli`, `randBinomial`, `randNegativeBinomial`, `randPoisson` ) to generate random values according to the specified distributions. This closes [#21834](https://github.com/ClickHouse/ClickHouse/issues/21834). [#42411](https://github.com/ClickHouse/ClickHouse/pull/42411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Added functions (`randUniform`, `randNormal`, `randLogNormal`, `randExponential`, `randChiSquared`, `randStudentT`, `randFisherF`, `randBernoulli`, `randBinomial`, `randNegativeBinomial`, `randPoisson`) to generate random values according to the specified distributions. This closes [#21834](https://github.com/ClickHouse/ClickHouse/issues/21834). [#42411](https://github.com/ClickHouse/ClickHouse/pull/42411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * An improvement for ClickHouse Keeper: add support for uploading snapshots to S3. S3 information can be defined inside `keeper_server.s3_snapshot`. [#41342](https://github.com/ClickHouse/ClickHouse/pull/41342) ([Antonio Andelic](https://github.com/antonio2368)). * Added an aggregate function `analysisOfVariance` (`anova`) to perform a statistical test over several groups of normally distributed observations to find out whether all groups have the same mean or not. Original PR [#37872](https://github.com/ClickHouse/ClickHouse/issues/37872). [#42131](https://github.com/ClickHouse/ClickHouse/pull/42131) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Support limiting of temporary data stored on disk using settings `max_temporary_data_on_disk_size_for_user`/`max_temporary_data_on_disk_size_for_query` . [#40893](https://github.com/ClickHouse/ClickHouse/pull/40893) ([Vladimir C](https://github.com/vdimir)). From 75885ce2e1883c2ef93b031985cfa512fbdced38 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Jul 2022 13:20:16 +0200 Subject: [PATCH 050/188] Added Analyzer --- src/Analyzer/CMakeLists.txt | 7 + src/Analyzer/ColumnNode.cpp | 70 + src/Analyzer/ColumnNode.h | 96 + src/Analyzer/ColumnTransformers.cpp | 302 +++ src/Analyzer/ColumnTransformers.h | 267 +++ src/Analyzer/ConstantNode.cpp | 56 + src/Analyzer/ConstantNode.h | 62 + src/Analyzer/FunctionNode.cpp | 123 + src/Analyzer/FunctionNode.h | 181 ++ src/Analyzer/IQueryTreeNode.cpp | 158 ++ src/Analyzer/IQueryTreeNode.h | 234 ++ src/Analyzer/IQueryTreePass.h | 38 + src/Analyzer/Identifier.h | 320 +++ src/Analyzer/IdentifierNode.cpp | 38 + src/Analyzer/IdentifierNode.h | 53 + src/Analyzer/InDepthQueryTreeVisitor.h | 67 + src/Analyzer/LambdaNode.cpp | 92 + src/Analyzer/LambdaNode.h | 116 + src/Analyzer/ListNode.cpp | 72 + src/Analyzer/ListNode.h | 48 + src/Analyzer/MatcherNode.cpp | 267 +++ src/Analyzer/MatcherNode.h | 171 ++ src/Analyzer/MultiIfToIfPass.cpp | 49 + src/Analyzer/MultiIfToIfPass.h | 25 + src/Analyzer/QueryAnalysisPass.cpp | 2024 +++++++++++++++++ src/Analyzer/QueryAnalysisPass.h | 49 + src/Analyzer/QueryNode.cpp | 100 + src/Analyzer/QueryNode.h | 118 + src/Analyzer/QueryTreeBuilder.cpp | 550 +++++ src/Analyzer/QueryTreeBuilder.h | 19 + src/Analyzer/QueryTreePassManager.cpp | 91 + src/Analyzer/QueryTreePassManager.h | 49 + src/Analyzer/TableNode.cpp | 60 + src/Analyzer/TableNode.h | 90 + src/Analyzer/examples/CMakeLists.txt | 3 + src/Analyzer/examples/query_analyzer.cpp | 11 + src/Analyzer/tests/CMakeLists.txt | 0 src/Analyzer/tests/gtest_identifier.cpp | 227 ++ src/Analyzer/tests/gtest_query_tree_node.cpp | 82 + src/CMakeLists.txt | 2 + src/Core/Settings.h | 1 + src/Functions/array/FunctionArrayMapped.h | 6 +- src/Functions/tupleElement.cpp | 28 +- src/Interpreters/InterpreterDescribeQuery.cpp | 19 +- src/Interpreters/InterpreterExplainQuery.cpp | 120 +- src/Interpreters/InterpreterFactory.cpp | 8 + .../InterpreterSelectQueryAnalyzer.cpp | 609 +++++ .../InterpreterSelectQueryAnalyzer.h | 36 + src/Parsers/ASTColumnsMatcher.cpp | 121 + src/Parsers/ASTColumnsMatcher.h | 35 + src/Parsers/ASTColumnsTransformers.cpp | 5 + src/Parsers/ASTColumnsTransformers.h | 1 + src/Parsers/ASTExplainQuery.h | 2 + src/Parsers/ASTFunction.h | 2 + src/Parsers/ASTIdentifier.h | 3 +- src/Parsers/ExpressionElementParsers.cpp | 164 +- src/Parsers/ExpressionElementParsers.h | 19 +- src/Parsers/ParserExplainQuery.cpp | 14 + src/Parsers/SelectUnionMode.cpp | 24 + src/Parsers/SelectUnionMode.h | 2 + 60 files changed, 7533 insertions(+), 73 deletions(-) create mode 100644 src/Analyzer/CMakeLists.txt create mode 100644 src/Analyzer/ColumnNode.cpp create mode 100644 src/Analyzer/ColumnNode.h create mode 100644 src/Analyzer/ColumnTransformers.cpp create mode 100644 src/Analyzer/ColumnTransformers.h create mode 100644 src/Analyzer/ConstantNode.cpp create mode 100644 src/Analyzer/ConstantNode.h create mode 100644 src/Analyzer/FunctionNode.cpp create mode 100644 src/Analyzer/FunctionNode.h create mode 100644 src/Analyzer/IQueryTreeNode.cpp create mode 100644 src/Analyzer/IQueryTreeNode.h create mode 100644 src/Analyzer/IQueryTreePass.h create mode 100644 src/Analyzer/Identifier.h create mode 100644 src/Analyzer/IdentifierNode.cpp create mode 100644 src/Analyzer/IdentifierNode.h create mode 100644 src/Analyzer/InDepthQueryTreeVisitor.h create mode 100644 src/Analyzer/LambdaNode.cpp create mode 100644 src/Analyzer/LambdaNode.h create mode 100644 src/Analyzer/ListNode.cpp create mode 100644 src/Analyzer/ListNode.h create mode 100644 src/Analyzer/MatcherNode.cpp create mode 100644 src/Analyzer/MatcherNode.h create mode 100644 src/Analyzer/MultiIfToIfPass.cpp create mode 100644 src/Analyzer/MultiIfToIfPass.h create mode 100644 src/Analyzer/QueryAnalysisPass.cpp create mode 100644 src/Analyzer/QueryAnalysisPass.h create mode 100644 src/Analyzer/QueryNode.cpp create mode 100644 src/Analyzer/QueryNode.h create mode 100644 src/Analyzer/QueryTreeBuilder.cpp create mode 100644 src/Analyzer/QueryTreeBuilder.h create mode 100644 src/Analyzer/QueryTreePassManager.cpp create mode 100644 src/Analyzer/QueryTreePassManager.h create mode 100644 src/Analyzer/TableNode.cpp create mode 100644 src/Analyzer/TableNode.h create mode 100644 src/Analyzer/examples/CMakeLists.txt create mode 100644 src/Analyzer/examples/query_analyzer.cpp create mode 100644 src/Analyzer/tests/CMakeLists.txt create mode 100644 src/Analyzer/tests/gtest_identifier.cpp create mode 100644 src/Analyzer/tests/gtest_query_tree_node.cpp create mode 100644 src/Interpreters/InterpreterSelectQueryAnalyzer.cpp create mode 100644 src/Interpreters/InterpreterSelectQueryAnalyzer.h create mode 100644 src/Parsers/SelectUnionMode.cpp diff --git a/src/Analyzer/CMakeLists.txt b/src/Analyzer/CMakeLists.txt new file mode 100644 index 00000000000..1068fee4cea --- /dev/null +++ b/src/Analyzer/CMakeLists.txt @@ -0,0 +1,7 @@ +if (ENABLE_TESTS) + add_subdirectory(tests) +endif() + +if (ENABLE_EXAMPLES) + add_subdirectory(examples) +endif() \ No newline at end of file diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp new file mode 100644 index 00000000000..3b112cf0909 --- /dev/null +++ b/src/Analyzer/ColumnNode.cpp @@ -0,0 +1,70 @@ +#include + +#include + +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +QueryTreeNodePtr ColumnNode::getColumnSource() const +{ + auto lock = column_source.lock(); + if (!lock) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column {} {} query tree node does not have valid source node", + column.name, + column.type->getName()); + + return lock; +} + +void ColumnNode::dumpTree(WriteBuffer & buffer, size_t indent) const +{ + buffer << std::string(indent, ' ') << "COLUMN "; + writePointerHex(this, buffer); + buffer << ' ' << column.name << " : " << column.type->getName() << " source "; + auto column_source_ptr = column_source.lock(); + writePointerHex(column_source_ptr.get(), buffer); +} + +void ColumnNode::updateTreeHashImpl(HashState & hash_state) const +{ + hash_state.update(column.name.size()); + hash_state.update(column.name); + + auto column_source_ptr = column_source.lock(); + if (column_source_ptr) + column_source_ptr->updateTreeHashImpl(hash_state); +} + +QueryTreeNodePtr ColumnNode::cloneImpl() const +{ + return std::make_shared(column, column_source); +} + +void ColumnNode::getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update) +{ + /** This method is called on node returned from `cloneImpl`. Check IQueryTreeNode.h interface. + * old pointer is current column source pointer. + * update place is address of column source. + */ + const auto * old_pointer = getColumnSource().get(); + pointers_to_update.emplace_back(old_pointer, &column_source); +} + +ASTPtr ColumnNode::toASTImpl() const +{ + return std::make_shared(column.name); +} + +} diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h new file mode 100644 index 00000000000..47f1bde48a3 --- /dev/null +++ b/src/Analyzer/ColumnNode.h @@ -0,0 +1,96 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +/** Column node represents column in query tree. + * Column must have some column source. + * Column can be table expression, lambda, subquery. + * Column source must be valid during column node lifetime. + * + * During query analysis pass identifier node is resolved into column. See IdentifierNode.h. + * + * Examples: + * SELECT id FROM test_table. id is identifier that must be resolved to column node during query analysis pass. + * SELECT lambda(x -> x + 1, [1,2,3]). x is identifier inside lambda that must be resolved to column node during query analysis pass. + * + * Column node is initialized with column name, type and column source weak pointer. + * Additional care must be taken during clone to repoint column source to another node if its necessary see IQueryTreeNode.h `clone` method. + */ +class ColumnNode; +using ColumnNodePtr = std::shared_ptr; + +class ColumnNode final : public IQueryTreeNode +{ +public: + /// Construct column node with column name, type and column source weak pointer. + explicit ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_) + : column(std::move(column_)) + , column_source(std::move(column_source_)) + {} + + /// Get column + const NameAndTypePair & getColumn() const + { + return column; + } + + /// Get column name + const String & getColumnName() const + { + return column.name; + } + + /// Get column type + const DataTypePtr & getColumnType() const + { + return column.type; + } + + /** Get column source. + * If column source is not valid logical exception is thrown. + */ + QueryTreeNodePtr getColumnSource() const; + + /// Get column source weak pointer + QueryTreeNodeWeakPtr getColumnSourceWeak() const + { + return column_source; + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::COLUMN; + } + + void dumpTree(WriteBuffer & buffer, size_t indent) const override; + + String getName() const override + { + return column.name; + } + + DataTypePtr getResultType() const override + { + return column.type; + } + +protected: + void updateTreeHashImpl(HashState & hash_state) const override; + + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + + void getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update) override; + +private: + NameAndTypePair column; + QueryTreeNodeWeakPtr column_source; +}; + +} diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp new file mode 100644 index 00000000000..7d51cb907b8 --- /dev/null +++ b/src/Analyzer/ColumnTransformers.cpp @@ -0,0 +1,302 @@ +#include + +#include + +#include +#include +#include + +#include +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +const char * toString(ColumnTransfomerType type) +{ + switch (type) + { + case ColumnTransfomerType::APPLY: return "APPLY"; + case ColumnTransfomerType::EXCEPT: return "EXCEPT"; + case ColumnTransfomerType::REPLACE: return "REPLACE"; + } +} + +/// ApplyColumnTransformerNode implementation + +const char * toString(ApplyColumnTransformerType type) +{ + switch (type) + { + case ApplyColumnTransformerType::LAMBDA: return "LAMBDA"; + case ApplyColumnTransformerType::FUNCTION: return "FUNCTION"; + } +} + +ApplyColumnTransformerNode::ApplyColumnTransformerNode(QueryTreeNodePtr expression_node_) +{ + if (expression_node_->getNodeType() == QueryTreeNodeType::LAMBDA) + apply_transformer_type = ApplyColumnTransformerType::LAMBDA; + else if (expression_node_->getNodeType() == QueryTreeNodeType::FUNCTION) + apply_transformer_type = ApplyColumnTransformerType::FUNCTION; + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Apply column transformer expression must be lambda or function. Actual {}", + expression_node_->getNodeTypeName()); + + children.resize(1); + children[expression_child_index] = std::move(expression_node_); +} + +void ApplyColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) const +{ + buffer << std::string(indent, ' ') << "APPLY COLUMN TRANSFORMER "; + writePointerHex(this, buffer); + buffer << ' ' << toString(apply_transformer_type) << '\n'; + + buffer << std::string(indent + 2, ' ') << "EXPRESSION" << '\n'; + + const auto & expression_node = getExpressionNode(); + expression_node->dumpTree(buffer, indent + 4); +} + +void ApplyColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const +{ + hash_state.update(static_cast(getTransformerType())); + getExpressionNode()->updateTreeHash(hash_state); +} + +ASTPtr ApplyColumnTransformerNode::toASTImpl() const +{ + auto ast_apply_transformer = std::make_shared(); + const auto & expression_node = getExpressionNode(); + + if (apply_transformer_type == ApplyColumnTransformerType::FUNCTION) + { + auto & function_expression = expression_node->as(); + ast_apply_transformer->func_name = function_expression.getFunctionName(); + ast_apply_transformer->parameters = function_expression.getParametersNode()->toAST(); + } + else + { + auto & lambda_expression = expression_node->as(); + if (!lambda_expression.getArgumentNames().empty()) + ast_apply_transformer->lambda_arg = lambda_expression.getArgumentNames()[0]; + ast_apply_transformer->lambda = lambda_expression.toAST(); + } + + return ast_apply_transformer; +} + +QueryTreeNodePtr ApplyColumnTransformerNode::cloneImpl() const +{ + ApplyColumnTransformerNodePtr result_apply_transformer(new ApplyColumnTransformerNode()); + return result_apply_transformer; +} + +/// ExceptColumnTransformerNode implementation + +bool ExceptColumnTransformerNode::isColumnMatching(const std::string & column_name) const +{ + if (column_matcher) + return RE2::PartialMatch(column_name, *column_matcher); + + for (const auto & name : except_column_names) + if (column_name == name) + return true; + + return false; +} + +const char * toString(ExceptColumnTransformerType type) +{ + switch (type) + { + case ExceptColumnTransformerType::REGEXP: + return "REGEXP"; + case ExceptColumnTransformerType::COLUMN_LIST: + return "COLUMN_LIST"; + } +} + +void ExceptColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) const +{ + buffer << std::string(indent, ' ') << "EXCEPT COLUMN TRANSFORMER "; + writePointerHex(this, buffer); + buffer << ' ' << toString(except_transformer_type) << ' '; + + if (column_matcher) + { + buffer << column_matcher->pattern(); + return; + } + + size_t except_column_names_size = except_column_names.size(); + for (size_t i = 0; i < except_column_names_size; ++i) + { + buffer << except_column_names[i]; + + if (i + 1 != except_column_names_size) + buffer << ", "; + } +} + +void ExceptColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const +{ + hash_state.update(static_cast(getTransformerType())); + hash_state.update(static_cast(getExceptTransformerType())); + + for (const auto & column_name : except_column_names) + { + hash_state.update(column_name.size()); + hash_state.update(column_name); + } + + if (column_matcher) + { + const auto & pattern = column_matcher->pattern(); + hash_state.update(pattern.size()); + hash_state.update(pattern); + } +} + +ASTPtr ExceptColumnTransformerNode::toASTImpl() const +{ + auto ast_except_transformer = std::make_shared(); + + if (column_matcher) + { + ast_except_transformer->setPattern(column_matcher->pattern()); + return ast_except_transformer; + } + + ast_except_transformer->children.reserve(except_column_names.size()); + for (const auto & name : except_column_names) + ast_except_transformer->children.push_back(std::make_shared(name)); + + return ast_except_transformer; +} + +QueryTreeNodePtr ExceptColumnTransformerNode::cloneImpl() const +{ + if (except_transformer_type == ExceptColumnTransformerType::REGEXP) + return std::make_shared(column_matcher); + + return std::make_shared(except_column_names); +} + +/// ReplaceColumnTransformerNode implementation + +ReplaceColumnTransformerNode::ReplaceColumnTransformerNode(const std::vector & replacements_) +{ + children.resize(1); + children[replacements_child_index] = std::make_shared(); + + auto & replacement_expressions_nodes = getReplacements().getNodes(); + + std::unordered_set replacement_names_set; + + for (const auto & replacement : replacements_) + { + auto [_, inserted] = replacement_names_set.emplace(replacement.column_name); + + if (!inserted) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Expressions in column transformer replace should not contain same replacement {} more than once", + replacement.column_name); + + replacements_names.push_back(replacement.column_name); + replacement_expressions_nodes.push_back(replacement.expression_node); + } +} + +QueryTreeNodePtr ReplaceColumnTransformerNode::findReplacementExpression(const std::string & expression_name) +{ + auto it = std::find(replacements_names.begin(), replacements_names.end(), expression_name); + if (it == replacements_names.end()) + return {}; + + size_t replacement_index = it - replacements_names.begin(); + auto & replacement_expressions_nodes = getReplacements().getNodes(); + return replacement_expressions_nodes[replacement_index]; +} + +void ReplaceColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) const +{ + buffer << std::string(indent, ' ') << "REPLACE TRANSFORMER "; + writePointerHex(this, buffer); + buffer << '\n'; + + auto & replacements_nodes = getReplacements().getNodes(); + size_t replacements_size = replacements_nodes.size(); + buffer << std::string(indent + 2, ' ') << "REPLACEMENTS " << replacements_size << '\n'; + + for (size_t i = 0; i < replacements_size; ++i) + { + const auto & replacement_name = replacements_names[i]; + buffer << std::string(indent + 4, ' ') << "REPLACEMENT NAME " << replacement_name; + buffer << " EXPRESSION" << '\n'; + const auto & expression_node = replacements_nodes[i]; + expression_node->dumpTree(buffer, indent + 6); + + if (i + 1 != replacements_size) + buffer << '\n'; + } +} + +void ReplaceColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const +{ + hash_state.update(static_cast(getTransformerType())); + + auto & replacement_expressions_nodes = getReplacements().getNodes(); + size_t replacements_size = replacement_expressions_nodes.size(); + hash_state.update(replacements_size); + + for (size_t i = 0; i < replacements_size; ++i) + { + const auto & replacement_name = replacements_names[i]; + hash_state.update(replacement_name.size()); + hash_state.update(replacement_name); + replacement_expressions_nodes[i]->updateTreeHash(hash_state); + } +} + +ASTPtr ReplaceColumnTransformerNode::toASTImpl() const +{ + auto ast_replace_transformer = std::make_shared(); + + auto & replacement_expressions_nodes = getReplacements().getNodes(); + size_t replacements_size = replacement_expressions_nodes.size(); + + ast_replace_transformer->children.reserve(replacements_size); + + for (size_t i = 0; i < replacements_size; ++i) + { + auto replacement_ast = std::make_shared(); + replacement_ast->name = replacements_names[i]; + replacement_ast->expr = replacement_expressions_nodes[i]->toAST(); + ast_replace_transformer->children.push_back(replacement_ast); + } + + return ast_replace_transformer; +} + +QueryTreeNodePtr ReplaceColumnTransformerNode::cloneImpl() const +{ + ReplaceColumnTransformerNodePtr result_replace_transformers(new ReplaceColumnTransformerNode()); + + result_replace_transformers->replacements_names = replacements_names; + + return result_replace_transformers; +} + +} diff --git a/src/Analyzer/ColumnTransformers.h b/src/Analyzer/ColumnTransformers.h new file mode 100644 index 00000000000..23c7a132271 --- /dev/null +++ b/src/Analyzer/ColumnTransformers.h @@ -0,0 +1,267 @@ +#pragma once + +#include + +#include +#include +#include + +namespace DB +{ + +/** Transformers are query tree nodes that handle additional logic that you can apply after MatcherQueryTreeNode is resolved. + * Check MatcherQueryTreeNode.h before reading this documentation. + * + * They main purpose it to apply some logic for expressions after matcher is resolved. + * There are 3 types of transformers: + * + * 1. APPLY transformer: + * APPLY transformer transform expression using lambda or function into another expression. + * It has 2 syntax variants: + * 1. lambda variant: SELECT matcher APPLY (x -> expr(x)). + * 2. function variant: SELECT matcher APPLY function_name(optional_parameters). + * + * 2. EXCEPT transformer: + * EXCEPT transformer discard some columns. + * It has 2 syntax variants: + * 1. regexp variant: SELECT matcher EXCEPT ('regexp'). + * 2. column names list variant: SELECT matcher EXCEPT (column_name_1, ...). + * + * 3. REPLACE transfomer: + * REPLACE transformer applies similar transformation as APPLY transformer, but only for expressions + * that match replacement expression name. + * + * Example: + * CREATE TABLE test_table (id UInt64) ENGINE=TinyLog; + * SELECT * REPLACE (id + 1 AS id) FROM test_table. + * This query is transformed into SELECT id + 1 FROM test_table. + * It is important that AS id is not alias, it is replacement name. id + 1 is replacement expression. + * + * REPLACE transformer cannot contain multiple replacements with same name. + * + * REPLACE transformer expression does not necessary include replacement column name. + * Example: + * SELECT * REPLACE (1 AS id) FROM test_table. + * + * REPLACE transformer expression does not throw exception if there are no columns to apply replacement. + * Example: + * SELECT * REPLACE (1 AS unknown_column) FROM test_table; + * + * REPLACE transform can contain multiple replacements. + * Example: + * SELECT * REPLACE (1 AS id, 2 AS value). + * + * Matchers can be combined together and chained. + * Example: + * SELECT * EXCEPT (id) APPLY (x -> toString(x)) APPLY (x -> length(x)) FROM test_table. + */ + +/// Column transformer type +enum class ColumnTransfomerType +{ + APPLY, + EXCEPT, + REPLACE +}; + +/// Get column transformer type name +const char * toString(ColumnTransfomerType type); + +class IColumnTransformerNode; +using ColumnTransformerNodePtr = std::shared_ptr; +using ColumnTransformersNodes = std::vector; + +/// IColumnTransformer base interface. +class IColumnTransformerNode : public IQueryTreeNode +{ +public: + + /// Get transformer type + virtual ColumnTransfomerType getTransformerType() const = 0; + + /// Get transformer type name + const char * getTransformerTypeName() const + { + return toString(getTransformerType()); + } + + QueryTreeNodeType getNodeType() const final + { + return QueryTreeNodeType::TRANSFORMER; + } +}; + +enum class ApplyColumnTransformerType +{ + LAMBDA, + FUNCTION +}; + +/// Get apply column transformer type name +const char * toString(ApplyColumnTransformerType type); + +class ApplyColumnTransformerNode; +using ApplyColumnTransformerNodePtr = std::shared_ptr; + +/// Apply column transformer +class ApplyColumnTransformerNode final : public IColumnTransformerNode +{ +public: + /** Initialize apply column transformer with expression node. + * Expression node must be lambda or function otherwise exception is throwned. + */ + explicit ApplyColumnTransformerNode(QueryTreeNodePtr expression_node_); + + /// Get apply transformer type + ApplyColumnTransformerType getApplyTransformerType() const + { + return apply_transformer_type; + } + + /// Get apply transformer expression node + const QueryTreeNodePtr & getExpressionNode() const + { + return children[expression_child_index]; + } + + ColumnTransfomerType getTransformerType() const override + { + return ColumnTransfomerType::APPLY; + } + + void dumpTree(WriteBuffer & buffer, size_t indent) const override; + +protected: + void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override; + + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + ApplyColumnTransformerNode() = default; + + ApplyColumnTransformerType apply_transformer_type = ApplyColumnTransformerType::LAMBDA; + static constexpr size_t expression_child_index = 0; +}; + +/// Except column transformer type +enum class ExceptColumnTransformerType +{ + REGEXP, + COLUMN_LIST, +}; + +const char * toString(ExceptColumnTransformerType type); + +class ExceptColumnTransformerNode; +using ExceptColumnTransformerNodePtr = std::shared_ptr; + +/// Except column transformer +class ExceptColumnTransformerNode final : public IColumnTransformerNode +{ +public: + /// Initialize except column transformer with column names + explicit ExceptColumnTransformerNode(Names except_column_names_) + : except_transformer_type(ExceptColumnTransformerType::COLUMN_LIST) + , except_column_names(std::move(except_column_names_)) + { + } + + /// Initialize except column transformer with regexp column matcher + explicit ExceptColumnTransformerNode(std::shared_ptr column_matcher_) + : except_transformer_type(ExceptColumnTransformerType::REGEXP) + , column_matcher(std::move(column_matcher_)) + { + } + + /// Get except transformer type + ExceptColumnTransformerType getExceptTransformerType() const + { + return except_transformer_type; + } + + /// Returns true if except transformer match column name, false otherwise. + bool isColumnMatching(const std::string & column_name) const; + + ColumnTransfomerType getTransformerType() const override + { + return ColumnTransfomerType::EXCEPT; + } + + void dumpTree(WriteBuffer & buffer, size_t indent) const override; + +protected: + void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override; + + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; +private: + ExceptColumnTransformerType except_transformer_type; + Names except_column_names; + std::shared_ptr column_matcher; +}; + +class ReplaceColumnTransformerNode; +using ReplaceColumnTransformerNodePtr = std::shared_ptr; + +/// Replace column transformer +class ReplaceColumnTransformerNode final : public IColumnTransformerNode +{ +public: + /// Replacement is column name and replace expression + struct Replacement + { + std::string column_name; + QueryTreeNodePtr expression_node; + }; + + /// Initialize replace column transformer with replacements + explicit ReplaceColumnTransformerNode(const std::vector & replacements_); + + ColumnTransfomerType getTransformerType() const override + { + return ColumnTransfomerType::REPLACE; + } + + /// Get replacements + ListNode & getReplacements() const + { + return children[replacements_child_index]->as(); + } + + /// Get replacements node + const QueryTreeNodePtr & getReplacementsNode() const + { + return children[replacements_child_index]; + } + + /// Get replacements names + const Names & getReplacementsNames() const + { + return replacements_names; + } + + /** Returns replacement expression if for expression name replacements exists, nullptr otherwise. + * Returned replacement expression must be cloned by caller. + */ + QueryTreeNodePtr findReplacementExpression(const std::string & expression_name); + + void dumpTree(WriteBuffer & buffer, size_t indent) const override; + +protected: + void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override; + + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + ReplaceColumnTransformerNode() = default; + + Names replacements_names; + static constexpr size_t replacements_child_index = 0; +}; + +} diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp new file mode 100644 index 00000000000..4e1824e17bc --- /dev/null +++ b/src/Analyzer/ConstantNode.cpp @@ -0,0 +1,56 @@ +#include + +#include +#include + +#include +#include +#include + +#include + +#include + +namespace DB +{ + +ConstantNode::ConstantNode(Field value_, DataTypePtr value_data_type_) + : value(std::move(value_)) + , value_string_dump(applyVisitor(FieldVisitorToString(), value)) + , type(std::move(value_data_type_)) +{} + +ConstantNode::ConstantNode(Field value_) + : value(std::move(value_)) + , value_string_dump(applyVisitor(FieldVisitorToString(), value)) + , type(applyVisitor(FieldToDataType(), value)) +{} + +void ConstantNode::dumpTree(WriteBuffer & buffer, size_t indent) const +{ + buffer << std::string(indent, ' ') << "CONSTANT "; + writePointerHex(this, buffer); + buffer << ' ' << value.dump() << " : " << type->getName(); +} + +void ConstantNode::updateTreeHashImpl(HashState & hash_state) const +{ + auto type_name = type->getName(); + hash_state.update(type_name.size()); + hash_state.update(type_name); + + hash_state.update(value_string_dump.size()); + hash_state.update(value_string_dump); +} + +ASTPtr ConstantNode::toASTImpl() const +{ + return std::make_shared(value); +} + +QueryTreeNodePtr ConstantNode::cloneImpl() const +{ + return std::make_shared(value, type); +} + +} diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h new file mode 100644 index 00000000000..44593a002f0 --- /dev/null +++ b/src/Analyzer/ConstantNode.h @@ -0,0 +1,62 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +/** Constant node represents constant value in query tree. + * Constant value must be representable by Field. + * Examples: 1, 'constant_string', [1,2,3]. + */ +class ConstantNode; +using ConstantNodePtr = std::shared_ptr; + +class ConstantNode final : public IQueryTreeNode +{ +public: + /// Construct constant query tree node from field and data type + explicit ConstantNode(Field value_, DataTypePtr value_data_type_); + + /// Construct constant query tree node from field, data type will be derived from field value + explicit ConstantNode(Field value_); + + /// Get constant value + const Field & getConstantValue() const + { + return value; + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::CONSTANT; + } + + void dumpTree(WriteBuffer & buffer, size_t indent) const override; + + String getName() const override + { + return value_string_dump; + } + + DataTypePtr getResultType() const override + { + return type; + } + +protected: + void updateTreeHashImpl(HashState & hash_state) const override; + + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + Field value; + String value_string_dump; + DataTypePtr type; +}; + +} diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp new file mode 100644 index 00000000000..3f8b1a73a12 --- /dev/null +++ b/src/Analyzer/FunctionNode.cpp @@ -0,0 +1,123 @@ +#include + +#include +#include + +#include +#include + +#include +#include + +#include + +namespace DB +{ + +void FunctionNode::resolveAsFunction(FunctionOverloadResolverPtr function_value, DataTypePtr result_type_value) +{ + aggregate_function = nullptr; + function = std::move(function_value); + result_type = std::move(result_type_value); + function_name = function->getName(); +} + +void FunctionNode::resolveAsAggregateFunction(AggregateFunctionPtr aggregate_function_value, DataTypePtr result_type_value) +{ + function = nullptr; + aggregate_function = std::move(aggregate_function_value); + result_type = std::move(result_type_value); + function_name = aggregate_function->getName(); +} + +void FunctionNode::dumpTree(WriteBuffer & buffer, size_t indent) const +{ + buffer << std::string(indent, ' ') << "FUNCTION "; + writePointerHex(this, buffer); + buffer << ' ' << function_name << (result_type ? (" : " + result_type->getName()) : ""); + + const auto & parameters = getParameters(); + if (!parameters.getNodes().empty()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "PARAMETERS\n"; + parameters.dumpTree(buffer, indent + 4); + } + + const auto & arguments = getArguments(); + if (!arguments.getNodes().empty()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "ARGUMENTS\n"; + arguments.dumpTree(buffer, indent + 4); + } +} + +String FunctionNode::getName() const +{ + String name = function_name; + + const auto & parameters = getParameters(); + const auto & parameters_nodes = parameters.getNodes(); + if (!parameters_nodes.empty()) + { + name += '('; + name += parameters.getName(); + name += ')'; + } + + const auto & arguments = getArguments(); + name += '('; + name += arguments.getName(); + name += ')'; + + return name; +} + +void FunctionNode::updateTreeHashImpl(HashState & hash_state) const +{ + hash_state.update(function_name.size()); + hash_state.update(function_name); + hash_state.update(isAggregateFunction()); + + if (result_type) + { + auto result_type_name = result_type->getName(); + hash_state.update(result_type_name.size()); + hash_state.update(result_type_name); + } +} + +ASTPtr FunctionNode::toASTImpl() const +{ + auto function_ast = std::make_shared(); + + function_ast->name = function_name; + + const auto & parameters = getParameters(); + if (!parameters.getNodes().empty()) + { + function_ast->children.push_back(parameters.toAST()); + function_ast->parameters = function_ast->children.back(); + } + + const auto & arguments = getArguments(); + if (!arguments.getNodes().empty()) + { + function_ast->children.push_back(arguments.toAST()); + function_ast->arguments = function_ast->children.back(); + } + + return function_ast; +} + +QueryTreeNodePtr FunctionNode::cloneImpl() const +{ + auto result_function = std::make_shared(function_name); + /// This is valid for clone method function or aggregate function must be stateless + result_function->function = function; + result_function->aggregate_function = aggregate_function; + result_function->result_type = result_type; + + return result_function; +} + +} diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h new file mode 100644 index 00000000000..d6215c7ebd5 --- /dev/null +++ b/src/Analyzer/FunctionNode.h @@ -0,0 +1,181 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class IFunctionOverloadResolver; +using FunctionOverloadResolverPtr = std::shared_ptr; + +class IAggregateFunction; +using AggregateFunctionPtr = std::shared_ptr; + +/** Function node represents function in query tree. + * Function syntax: function_name(parameter_1, ...)(argument_1, ...). + * If funciton does not have parameters its syntax is function_name(argument_1, ...). + * If function does not have arguments its syntax is function_name(). + * + * In query tree function parameters and arguments are represented by ListNode. + * + * Function can be: + * 1. Aggregate function. Example: quantile(0.5)(x), sum(x). + * 2. Non aggregate function. Example: plus(x, x). + * + * Initially function node is initialize with function name. + * During query analysis pass function must be resolved using `resolveAsFunction` or `resolveAsAggregateFunction` methods. + * Resolved function is function that has result type and is initialized with concrete aggregate or non aggregate function. + */ +class FunctionNode; +using FunctionNodePtr = std::shared_ptr; + +class FunctionNode final : public IQueryTreeNode +{ +public: + /** Initialize function node with function name. + * Later during query analysis path function must be resolved. + */ + explicit FunctionNode(String function_name_) + : function_name(function_name_) + { + children.resize(2); + children[parameters_child_index] = std::make_shared(); + children[arguments_child_index] = std::make_shared(); + } + + /// Get name + const String & getFunctionName() const + { + return function_name; + } + + /// Get parameters + const ListNode & getParameters() const + { + return children[parameters_child_index]->as(); + } + + /// Get parameters + ListNode & getParameters() + { + return children[parameters_child_index]->as(); + } + + /// Get parameters node + const QueryTreeNodePtr & getParametersNode() const + { + return children[parameters_child_index]; + } + + /// Get parameters node + QueryTreeNodePtr & getParametersNode() + { + return children[parameters_child_index]; + } + + /// Get arguments + const ListNode & getArguments() const + { + return children[arguments_child_index]->as(); + } + + /// Get arguments + ListNode & getArguments() + { + return children[arguments_child_index]->as(); + } + + /// Get arguments node + const QueryTreeNodePtr & getArgumentsNode() const + { + return children[arguments_child_index]; + } + + /// Get arguments node + QueryTreeNodePtr & getArgumentsNode() + { + return children[arguments_child_index]; + } + + /** Get non aggregate function. + * If function is not resolved nullptr returned. + */ + const FunctionOverloadResolverPtr & getFunction() const + { + return function; + } + + /** Get aggregate function. + * If function is not resolved nullptr returned. + * If function is resolved as non aggregate function nullptr returned. + */ + const AggregateFunctionPtr & getAggregateFunction() const + { + return aggregate_function; + } + + /// Is function node resolved + bool isResolved() const + { + return function != nullptr || aggregate_function != nullptr; + } + + /// Is function node resolved as aggregate function + bool isAggregateFunction() const + { + return aggregate_function != nullptr; + } + + /// Is function node resolved as non aggregate function + bool isNonAggregateFunction() const + { + return function != nullptr; + } + + /** Resolve function node as non aggregate function. + * It is important that function name is update with resolved function name. + * Main motiviation for this is query tree optimizations. + * Assume we have `multiIf` function with single argument, it can be converted to `if` function. + * Function name must be updated accordingly. + */ + void resolveAsFunction(FunctionOverloadResolverPtr function_value, DataTypePtr result_type_value); + + /** Resolve function node as aggregate function. + * It is important that function name is update with resolved function name. + * Main motiviation for this is query tree optimizations. + */ + void resolveAsAggregateFunction(AggregateFunctionPtr aggregate_function_value, DataTypePtr result_type_value); + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::FUNCTION; + } + + void dumpTree(WriteBuffer & buffer, size_t indent) const override; + + DataTypePtr getResultType() const override + { + return result_type; + } + + String getName() const override; + +protected: + void updateTreeHashImpl(HashState & hash_state) const override; + + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + static constexpr size_t parameters_child_index = 0; + static constexpr size_t arguments_child_index = 1; + + String function_name; + FunctionOverloadResolverPtr function; + AggregateFunctionPtr aggregate_function; + DataTypePtr result_type; +}; + +} diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp new file mode 100644 index 00000000000..8a84ad2a9df --- /dev/null +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -0,0 +1,158 @@ +#include + +#include + +#include + +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + +const char * toString(QueryTreeNodeType type) +{ + switch (type) + { + case QueryTreeNodeType::IDENTIFIER: return "IDENTIFIER"; + case QueryTreeNodeType::ASTERISK: return "ASTERISK"; + case QueryTreeNodeType::TRANSFORMER: return "TRANSFORMER"; + case QueryTreeNodeType::LIST: return "LIST"; + case QueryTreeNodeType::CONSTANT: return "CONSTANT"; + case QueryTreeNodeType::FUNCTION: return "FUNCTION"; + case QueryTreeNodeType::COLUMN: return "COLUMN"; + case QueryTreeNodeType::LAMBDA: return "LAMBDA"; + case QueryTreeNodeType::TABLE: return "TABLE"; + case QueryTreeNodeType::QUERY: return "QUERY"; + } +} + +String IQueryTreeNode::dumpTree() const +{ + WriteBufferFromOwnString buff; + dumpTree(buff, 0); + return buff.str(); +} + +IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const +{ + HashState hash_state; + updateTreeHash(hash_state); + + Hash result; + hash_state.get128(result); + + return result; +} + +void IQueryTreeNode::updateTreeHash(HashState & state) const +{ + updateTreeHashImpl(state); + state.update(children.size()); + + for (const auto & child : children) + { + if (!child) + continue; + + child->updateTreeHash(state); + } +} + +QueryTreeNodePtr IQueryTreeNode::clone() const +{ + /** Main motivation for this method is to allow nodes in query tree have weak pointers to other nodes. + * Main use cases is for column node to have weak pointer to its source. + * Source can be lambda, table, subquery and such information is useful for later analysis stages. + * + * Algorithm + * For each node we clone state and also create mapping old_pointer to new pointer. + * For each cloned node we also update node_pointer_to_update_node_pointers array. + * + * After that we can update pointer in node_pointer_to_update_node_pointers using old_pointer to new pointer mapping. + */ + std::unordered_map old_pointer_to_new_pointer; + QueryTreePointersToUpdate pointers_to_update_after_clone; + + QueryTreeNodePtr result_cloned_node_place; + + std::deque> nodes_to_clone; + nodes_to_clone.emplace_back(this, &result_cloned_node_place); + + while (!nodes_to_clone.empty()) + { + const auto [node_to_clone, cloned_node_place] = nodes_to_clone.front(); + nodes_to_clone.pop_front(); + + auto node_clone = node_to_clone->cloneImpl(); + *cloned_node_place = node_clone; + + node_clone->setAlias(node_to_clone->alias); + node_clone->setOriginalAST(node_to_clone->original_ast); + node_clone->children = node_to_clone->children; + + node_clone->getPointersToUpdateAfterClone(pointers_to_update_after_clone); + old_pointer_to_new_pointer.emplace(node_to_clone, node_clone); + + for (auto & child : node_clone->children) + { + if (!child) + continue; + + nodes_to_clone.emplace_back(child.get(), &child); + } + } + + for (auto & [old_pointer, new_pointer] : pointers_to_update_after_clone) + { + auto it = old_pointer_to_new_pointer.find(old_pointer); + + /** If node had weak pointer to some other node and this node is not valid in cloned subtree part do not clone it. + * It will continue to point to previous location and it is expected. + * + * For example: SELECT id as a, a FROM test_table. + * id is resolved as column and test_table is source. + * a is resolved as id and after resolve must be cloned. + * weak pointer to source from a will point to old id location. + */ + if (it == old_pointer_to_new_pointer.end()) + continue; + + *new_pointer = it->second; + } + + return result_cloned_node_place; +} + +ASTPtr IQueryTreeNode::toAST() const +{ + auto converted_node = toASTImpl(); + + if (auto * ast_with_alias = typeid_cast(converted_node.get())) + converted_node->setAlias(alias); + + return converted_node; +} + +String IQueryTreeNode::formatOriginalASTForErrorMessage() const +{ + if (!original_ast) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Original AST was not set"); + + return original_ast->formatForErrorMessage(); +} + +String IQueryTreeNode::formatConvertedASTForErrorMessage() const +{ + return toAST()->formatForErrorMessage(); +} + +} diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h new file mode 100644 index 00000000000..df99e0e2b90 --- /dev/null +++ b/src/Analyzer/IQueryTreeNode.h @@ -0,0 +1,234 @@ +#pragma once + +#include +#include +#include + +#include + +#include + +#include + +#include + +class SipHash; + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + +class WriteBuffer; + +/// Query tree node type +enum class QueryTreeNodeType +{ + IDENTIFIER, + ASTERISK, + TRANSFORMER, + LIST, + CONSTANT, + FUNCTION, + COLUMN, + LAMBDA, + TABLE, + QUERY, +}; + +/// Convert query tree node type to string +const char * toString(QueryTreeNodeType type); + +/** Query tree node represent node in query tree. + * This is base class for all query tree nodes. + */ +class IQueryTreeNode; +using QueryTreeNodePtr = std::shared_ptr; +using QueryTreeNodeWeakPtr = std::weak_ptr; +using QueryTreeNodes = std::vector; + +class IQueryTreeNode : public TypePromotion +{ +public: + virtual ~IQueryTreeNode() = default; + + /// Get query tree node type + virtual QueryTreeNodeType getNodeType() const = 0; + + const char * getNodeTypeName() const + { + return toString(getNodeType()); + } + + /** Get name of query tree node that can be used as part of expression. + * TODO: Projection name, expression name must be refactored in better interface. + */ + virtual String getName() const + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method is not supported for {} query node", getNodeTypeName()); + } + + /** Get result type of query tree node that can be used as part of expression. + * If node does not support this method exception is throwed. + * TODO: Maybe this can be a part of ExpressionQueryTreeNode. + */ + virtual DataTypePtr getResultType() const + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method is not supported for {} query node", getNodeTypeName()); + } + + /// Dump query tree to string + String dumpTree() const; + + /// Dump query tree to buffer + void dumpTree(WriteBuffer & buffer) const + { + dumpTree(buffer, 0); + } + + /// Dump query tree to buffer starting with indent + virtual void dumpTree(WriteBuffer & buffer, size_t indent) const = 0; + + using Hash = std::pair; + using HashState = SipHash; + + /// Get tree hash identifying current tree + Hash getTreeHash() const; + + /// Update tree hash + void updateTreeHash(HashState & state) const; + + /// Get a deep copy of the query tree + QueryTreeNodePtr clone() const; + + /// Check if node has alias + bool hasAlias() const + { + return !alias.empty(); + } + + /// Get node alias value if specified + const String & getAlias() const + { + return alias; + } + + /// Set node alias value + void setAlias(String alias_value) + { + alias = std::move(alias_value); + } + + /// Remove node alias value + void removeAlias() + { + alias = {}; + } + + /// Check if query tree node has original AST + bool hasOriginalAST() const + { + return original_ast != nullptr; + } + + /// Get query tree node original AST + const ASTPtr & getOriginalAST() const + { + return original_ast; + } + + /** Set query tree node original AST. + * This AST will not be modified later. + */ + void setOriginalAST(ASTPtr original_ast_value) + { + original_ast = std::move(original_ast_value); + } + + /** If query tree has original AST format it for error message. + * Otherwise throw an exception. + */ + String formatOriginalASTForErrorMessage() const; + + /// Convert query tree to AST + ASTPtr toAST() const; + + /// Convert query tree to AST and then format it for error message. + String formatConvertedASTForErrorMessage() const; + + /** Format AST for error message. + * If original AST exists use `formatOriginalASTForErrorMessage`. + * Otherwise use `formatConvertedASTForErrorMessage`. + */ + String formatASTForErrorMessage() const + { + if (original_ast) + return formatOriginalASTForErrorMessage(); + + return formatConvertedASTForErrorMessage(); + } + + /// Get query tree node children + QueryTreeNodes & getChildren() + { + return children; + } + + /// Get query tree node children + const QueryTreeNodes & getChildren() const + { + return children; + } + + /** Subclass must update tree hash of its internal state and do not update tree hash for children. + * Caller must update tree hash for node children. + * + * This method is not protected because if subclass node has weak pointers to other query tree nodes it must use it + * as part of its updateTreeHashImpl method. In child classes this method should be protected. + */ + virtual void updateTreeHashImpl(HashState & hash_state) const = 0; + +protected: + /** Subclass node must convert itself to AST. + * Subclass must convert children to AST. + */ + virtual ASTPtr toASTImpl() const = 0; + + /** Subclass must clone only it internal state. + * Subclass children will be cloned separately by caller. + */ + virtual QueryTreeNodePtr cloneImpl() const = 0; + + /** If node has weak pointers to other tree nodes during clone they will point to other tree nodes. + * Keeping weak pointer to other tree nodes can be useful for example for column to keep weak pointer to column source. + * Source can be table, lambda, subquery and such information is necessary to preserve. + * + * Example: + * SELECT id FROM table; + * id during query analysis will be resolved as ColumnNode and source will be TableNode. + * During clone we must update id ColumnNode source pointer. + * + * Subclass must save old pointer and place of pointer update into pointers_to_update. + * This method will be called on query tree node after clone. + * + * Root of clone process will update pointers as necessary. + */ + using QueryTreePointerToUpdate = std::pair; + using QueryTreePointersToUpdate = std::vector; + + virtual void getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update) + { + (void)(pointers_to_update); + } + + QueryTreeNodes children; + +private: + String alias; + ASTPtr original_ast; +}; + +} diff --git a/src/Analyzer/IQueryTreePass.h b/src/Analyzer/IQueryTreePass.h new file mode 100644 index 00000000000..9c565438dde --- /dev/null +++ b/src/Analyzer/IQueryTreePass.h @@ -0,0 +1,38 @@ +#pragma once + +#include + +#include + + +namespace DB +{ + +/** After query tree is build it can be later processed by QueryTreePassManager. + * This is abstract base class for all query tree passes. + * + * Query tree pass can make query tree modifications, after each pass query tree must be valid. + * Query tree pass must perform be isolated and perform only necessary query tree modifications for doing its job. + * Dependencies between passes must be avoided. + */ +class IQueryTreePass; +using QueryTreePassPtr = std::shared_ptr; +using QueryTreePasses = std::vector; + +class IQueryTreePass +{ +public: + virtual ~IQueryTreePass() = default; + + /// Get query tree pass name + virtual String getName() = 0; + + /// Get query tree pass description + virtual String getDescription() = 0; + + /// Run pass over query tree + virtual void run(QueryTreeNodePtr query_tree_node, ContextPtr context) = 0; + +}; + +} diff --git a/src/Analyzer/Identifier.h b/src/Analyzer/Identifier.h new file mode 100644 index 00000000000..654c7021cf5 --- /dev/null +++ b/src/Analyzer/Identifier.h @@ -0,0 +1,320 @@ +#pragma once + +#include +#include + +#include +#include + + +namespace DB +{ + +/** Identifier constists from identifier parts. + * Each identifier part is arbitrary long sequence of digits, underscores, lowercase and uppercase letters. + * Example: a, a.b, a.b.c. + */ +class Identifier +{ +public: + Identifier() = default; + + /// Create Identifier from parts + explicit Identifier(const std::vector & parts_) + : full_name(boost::algorithm::join(parts_, ".")) + , parts(parts_) + { + } + + /// Create Identifier from full_name. Full_name is splitted with '.' as separator. + explicit Identifier(const std::string & full_name_) + : full_name(full_name_) + { + boost::split(parts, full_name, [](char c) { return c == '.'; }); + } + + const std::string & getFullName() const + { + return full_name; + } + + const std::vector & getParts() const + { + return parts; + } + + size_t getPartsSize() const + { + return parts.size(); + } + + bool empty() const + { + return parts.empty(); + } + + bool isEmpty() const + { + return parts.empty(); + } + + bool isShort() const + { + return parts.size() == 1; + } + + bool isCompound() const + { + return parts.size() > 1; + } + + const std::string & at(size_t index) const + { + if (index >= parts.size()) + throw std::out_of_range("identifier access part is out of range"); + + return parts[index]; + } + + const std::string & operator[](size_t index) const + { + return parts[index]; + } + + const std::string & front() const + { + return parts.front(); + } + + const std::string & back() const + { + return parts.back(); + } + + /// Returns true, if identifier starts with part, false otherwise + bool startsWith(const std::string_view & part) + { + return !parts.empty() && parts[0] == part; + } + + /// Returns true, if identifier ends with part, false otherwise + bool endsWith(const std::string_view & part) + { + return !parts.empty() && parts.back() == part; + } + + using const_iterator = std::vector::const_iterator; + + const_iterator begin() const + { + return parts.begin(); + } + + const_iterator end() const + { + return parts.end(); + } + + void popFirst(size_t parts_to_remove_size) + { + assert(parts_to_remove_size <= parts.size()); + + size_t parts_size = parts.size(); + std::vector result_parts; + result_parts.reserve(parts_size - parts_to_remove_size); + + for (size_t i = parts_to_remove_size; i < parts_size; ++i) + result_parts.push_back(std::move(parts[i])); + + parts = std::move(result_parts); + full_name = boost::algorithm::join(parts, "."); + } + + void popFirst() + { + return popFirst(1); + } + + void popLast(size_t parts_to_remove_size) + { + assert(parts_to_remove_size <= parts.size()); + + for (size_t i = 0; i < parts_to_remove_size; ++i) + parts.pop_back(); + + full_name = boost::algorithm::join(parts, "."); + } + + void popLast() + { + return popLast(1); + } + +private: + std::string full_name; + std::vector parts; +}; + +inline bool operator==(const Identifier & lhs, const Identifier & rhs) +{ + return lhs.getFullName() == rhs.getFullName(); +} + +inline bool operator!=(const Identifier & lhs, const Identifier & rhs) +{ + return !(lhs == rhs); +} + +inline std::ostream & operator<<(std::ostream & stream, const Identifier & identifier) +{ + stream << identifier.getFullName(); + return stream; +} + +using Identifiers = std::vector; + +/// View for Identifier +class IdentifierView +{ +public: + IdentifierView() = default; + + IdentifierView(const Identifier & identifier) /// NOLINT + : full_name_view(identifier.getFullName()) + , parts_start_it(identifier.begin()) + , parts_end_it(identifier.end()) + {} + + std::string_view getFullName() const + { + return full_name_view; + } + + size_t getPartsSize() const + { + return parts_end_it - parts_start_it; + } + + bool empty() const + { + return parts_start_it == parts_end_it; + } + + bool isEmpty() const + { + return parts_start_it == parts_end_it; + } + + bool isShort() const + { + return getPartsSize() == 1; + } + + bool isCompound() const + { + return getPartsSize() > 1; + } + + std::string_view at(size_t index) const + { + if (index >= getPartsSize()) + throw std::out_of_range("identifier access part is out of range"); + + return *(parts_start_it + index); + } + + std::string_view operator[](size_t index) const + { + return *(parts_start_it + index); + } + + std::string_view front() const + { + return *parts_start_it; + } + + std::string_view back() const + { + return *(parts_end_it - 1); + } + + bool startsWith(std::string_view part) const + { + return !isEmpty() && *parts_start_it == part; + } + + bool endsWith(std::string_view part) const + { + return !isEmpty() && *(parts_end_it - 1) == part; + } + + void popFirst(size_t parts_to_remove_size) + { + assert(parts_to_remove_size <= getPartsSize()); + + for (size_t i = 0; i < parts_to_remove_size; ++i) + { + size_t part_size = parts_start_it->size(); + ++parts_start_it; + bool is_not_last = parts_start_it != parts_end_it; + full_name_view.remove_prefix(part_size + is_not_last); + } + } + + void popFirst() + { + popFirst(1); + } + + void popLast(size_t parts_to_remove_size) + { + assert(parts_to_remove_size <= getPartsSize()); + + for (size_t i = 0; i < parts_to_remove_size; ++i) + { + size_t last_part_size = (parts_end_it - 1)->size(); + --parts_end_it; + bool is_not_last = parts_start_it != parts_end_it; + full_name_view.remove_suffix(last_part_size + is_not_last); + } + } + + void popLast() + { + popLast(1); + } + + using const_iterator = Identifier::const_iterator; + + const_iterator begin() const + { + return parts_start_it; + } + + const_iterator end() const + { + return parts_end_it; + } +private: + std::string_view full_name_view; + const_iterator parts_start_it; + const_iterator parts_end_it; +}; + +inline bool operator==(const IdentifierView & lhs, const IdentifierView & rhs) +{ + return lhs.getFullName() == rhs.getFullName(); +} + +inline bool operator!=(const IdentifierView & lhs, const IdentifierView & rhs) +{ + return !(lhs == rhs); +} + +inline std::ostream & operator<<(std::ostream & stream, const IdentifierView & identifier_view) +{ + stream << identifier_view.getFullName(); + return stream; +} + +} diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp new file mode 100644 index 00000000000..06892344dd0 --- /dev/null +++ b/src/Analyzer/IdentifierNode.cpp @@ -0,0 +1,38 @@ +#include + +#include + +#include +#include + +#include + +namespace DB +{ + +void IdentifierNode::dumpTree(WriteBuffer & buffer, size_t indent) const +{ + buffer << std::string(indent, ' ') << "IDENTIFIER "; + writePointerHex(this, buffer); + buffer << ' ' << identifier.getFullName(); +} + +void IdentifierNode::updateTreeHashImpl(HashState & state) const +{ + const auto & identifier_name = identifier.getFullName(); + state.update(identifier_name.size()); + state.update(identifier_name); +} + +ASTPtr IdentifierNode::toASTImpl() const +{ + auto identifier_parts = identifier.getParts(); + return std::make_shared(std::move(identifier_parts)); +} + +QueryTreeNodePtr IdentifierNode::cloneImpl() const +{ + return std::make_shared(identifier); +} + +} diff --git a/src/Analyzer/IdentifierNode.h b/src/Analyzer/IdentifierNode.h new file mode 100644 index 00000000000..054a8d323bb --- /dev/null +++ b/src/Analyzer/IdentifierNode.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/** Identifier node represents identifier in query tree. + * Example: SELECT a FROM test_table. + * a - is identifier. + * test_table - is identifier. + * + * Identifier resolution must be done during query analysis pass. + */ +class IdentifierNode final : public IQueryTreeNode +{ +public: + /// Construct identifier node with identifier + explicit IdentifierNode(Identifier identifier_) + : identifier(std::move(identifier_)) + {} + + /// Get identifier + const Identifier & getIdentifier() const + { + return identifier; + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::IDENTIFIER; + } + + void dumpTree(WriteBuffer & buffer, size_t indent) const override; + + String getName() const override + { + return identifier.getFullName(); + } + +protected: + void updateTreeHashImpl(HashState & state) const override; + + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + Identifier identifier; +}; + +} diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h new file mode 100644 index 00000000000..3eba5f2b15c --- /dev/null +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -0,0 +1,67 @@ +#pragma once + +#include + +#include + + +namespace DB +{ + +/** Visit query tree in depth. + * Matcher need to define `visit`, `needChildVisit` methods and `Data` type. + */ +template +class InDepthQueryTreeVisitor +{ +public: + using Data = typename Matcher::Data; + + /// Initialize visitor with matchers data + explicit InDepthQueryTreeVisitor(Data & data_) + : data(data_) + {} + + /// Visit query tree node + void visit(QueryTreeNodePtr & query_tree_node) + { + if constexpr (!top_to_bottom) + visitChildren(query_tree_node); + + try + { + Matcher::visit(query_tree_node, data); + } + catch (Exception & e) + { + e.addMessage("While processing {}", query_tree_node->formatASTForErrorMessage()); + throw; + } + + if constexpr (top_to_bottom) + visitChildren(query_tree_node); + } + +private: + Data & data; + + void visitChildren(QueryTreeNodePtr & expression) + { + for (auto & child : expression->getChildren()) + { + if (!child) + continue; + + bool need_visit_child = false; + if constexpr (need_child_accept_data) + need_visit_child = Matcher::needChildVisit(expression, child, data); + else + need_visit_child = Matcher::needChildVisit(expression, child); + + if (need_visit_child) + visit(child); + } + } +}; + +} diff --git a/src/Analyzer/LambdaNode.cpp b/src/Analyzer/LambdaNode.cpp new file mode 100644 index 00000000000..12c6c7992ff --- /dev/null +++ b/src/Analyzer/LambdaNode.cpp @@ -0,0 +1,92 @@ +#include + +#include + +#include +#include +#include + +namespace DB +{ + +LambdaNode::LambdaNode(Names argument_names_, QueryTreeNodePtr expression_) + : argument_names(std::move(argument_names_)) +{ + children.resize(2); + + auto arguments_list_node = std::make_shared(); + auto & nodes = arguments_list_node->getNodes(); + + size_t argument_names_size = argument_names.size(); + nodes.reserve(argument_names_size); + + for (size_t i = 0; i < argument_names_size; ++i) + nodes.push_back(std::make_shared(Identifier{argument_names[i]})); + + children[arguments_child_index] = std::move(arguments_list_node); + children[expression_child_index] = std::move(expression_); +} + +void LambdaNode::dumpTree(WriteBuffer & buffer, size_t indent) const +{ + auto result_type = getExpression()->getResultType(); + + buffer << std::string(indent, ' ') << "LAMBDA "; + writePointerHex(this, buffer); + buffer << (result_type ? (" : " + result_type->getName()) : "") << '\n'; + + buffer << std::string(indent + 2, ' ') << "ARGUMENTS " << '\n'; + getArguments().dumpTree(buffer, indent + 4); + + buffer << '\n'; + + buffer << std::string(indent + 2, ' ') << "EXPRESSION " << '\n'; + getExpression()->dumpTree(buffer, indent + 4); +} + +String LambdaNode::getName() const +{ + return "lambda(" + children[arguments_child_index]->getName() + ") -> " + children[expression_child_index]->getName(); +} + +void LambdaNode::updateTreeHashImpl(HashState & state) const +{ + state.update(argument_names.size()); + for (const auto & argument_name : argument_names) + { + state.update(argument_name.size()); + state.update(argument_name); + } +} + +ASTPtr LambdaNode::toASTImpl() const +{ + auto lambda_function_arguments_ast = std::make_shared(); + + auto tuple_function = std::make_shared(); + tuple_function->name = "tuple"; + tuple_function->children.push_back(children[arguments_child_index]->toAST()); + tuple_function->arguments = tuple_function->children.back(); + + lambda_function_arguments_ast->children.push_back(std::move(tuple_function)); + lambda_function_arguments_ast->children.push_back(children[expression_child_index]->toAST()); + + auto lambda_function_ast = std::make_shared(); + lambda_function_ast->name = "lambda"; + lambda_function_ast->is_lambda_function = true; + lambda_function_ast->children.push_back(std::move(lambda_function_arguments_ast)); + lambda_function_ast->arguments = lambda_function_ast->children.back(); + + return lambda_function_ast; +} + +QueryTreeNodePtr LambdaNode::cloneImpl() const +{ + LambdaNodePtr result_lambda(new LambdaNode()); + + result_lambda->argument_names = argument_names; + + return result_lambda; +} + +} diff --git a/src/Analyzer/LambdaNode.h b/src/Analyzer/LambdaNode.h new file mode 100644 index 00000000000..3d07ae0583e --- /dev/null +++ b/src/Analyzer/LambdaNode.h @@ -0,0 +1,116 @@ +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ + +/** Lambda node represents lambda expression in query tree. + * + * Lambda consist of argument names and lambda expression body. + * Lambda expression body does not necessary use lambda arguments. Example: SELECT arrayMap(x -> 1, [1, 2, 3]) + * + * Initially lambda is initialized with argument names and expression query tree node. + * During query analysis if expression is not resolved lambda must be resolved. + * Lambda is resolved if lambda expression is resolved. + * + * It is important that lambda expression result type can depend on arguments types. + * Example: WITH (x -> x) as lambda SELECT lambda(1), lambda('string_value'). + * + * During query analysis pass lambdas must be resolved. + * Lambda resolve must set concrete lambda arguments and resolve lambda expression body. + * In query tree lambda arguments are represented by ListNode. + * If client modified lambda arguments array its size must be equal to initial lambda argument names array. + * + * Examples: + * WITH (x -> x + 1) as lambda SELECT lambda(1). + * SELECT arrayMap(x -> x + 1, [1,2,3]). + */ +class LambdaNode; +using LambdaNodePtr = std::shared_ptr; + +class LambdaNode final : public IQueryTreeNode +{ +public: + /// Initialize lambda with argument names and expression query tree node + explicit LambdaNode(Names argument_names_, QueryTreeNodePtr expression_); + + /// Get argument names + const Names & getArgumentNames() const + { + return argument_names; + } + + /// Get arguments + const ListNode & getArguments() const + { + return children[arguments_child_index]->as(); + } + + /// Get arguments + ListNode & getArguments() + { + return children[arguments_child_index]->as(); + } + + /// Get arguments node + const QueryTreeNodePtr & getArgumentsNode() const + { + return children[arguments_child_index]; + } + + /** Get arguments node. + * If arguments array is modified its result size must be equal to lambd argument names size. + */ + QueryTreeNodePtr & getArgumentsNode() + { + return children[arguments_child_index]; + } + + /// Get expression + const QueryTreeNodePtr & getExpression() const + { + return children[expression_child_index]; + } + + /// Get expression + QueryTreeNodePtr & getExpression() + { + return children[expression_child_index]; + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::LAMBDA; + } + + void dumpTree(WriteBuffer & buffer, size_t indent) const override; + + String getName() const override; + + DataTypePtr getResultType() const override + { + return getExpression()->getResultType(); + } + +protected: + void updateTreeHashImpl(HashState & state) const override; + + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + LambdaNode() = default; + + static constexpr size_t arguments_child_index = 0; + static constexpr size_t expression_child_index = 1; + + Names argument_names; +}; + +} diff --git a/src/Analyzer/ListNode.cpp b/src/Analyzer/ListNode.cpp new file mode 100644 index 00000000000..b8c6a974101 --- /dev/null +++ b/src/Analyzer/ListNode.cpp @@ -0,0 +1,72 @@ +#include + +#include + +#include +#include +#include + +#include + +namespace DB +{ + +void ListNode::dumpTree(WriteBuffer & buffer, size_t indent) const +{ + size_t children_size = children.size(); + buffer << std::string(indent, ' ') << "LIST "; + writePointerHex(this, buffer); + buffer << ' ' << children_size << '\n'; + + for (size_t i = 0; i < children_size; ++i) + { + const auto & node = children[i]; + node->dumpTree(buffer, indent + 2); + + if (i + 1 != children_size) + buffer << '\n'; + } +} + +String ListNode::getName() const +{ + if (children.empty()) + return ""; + + std::string result; + for (const auto & node : children) + { + result += node->getName(); + result += ", "; + } + + result.pop_back(); + result.pop_back(); + + return result; +} + +void ListNode::updateTreeHashImpl(HashState &) const +{ + /// Hash automatically updated with children size in IQueryTreeNode.h +} + +ASTPtr ListNode::toASTImpl() const +{ + auto expression_list_ast = std::make_shared(); + + size_t children_size = children.size(); + expression_list_ast->children.resize(children_size); + + for (size_t i = 0; i < children_size; ++i) + expression_list_ast->children[i] = children[i]->toAST(); + + return expression_list_ast; +} + +QueryTreeNodePtr ListNode::cloneImpl() const +{ + return std::make_shared(); +} + +} diff --git a/src/Analyzer/ListNode.h b/src/Analyzer/ListNode.h new file mode 100644 index 00000000000..9071c7c275f --- /dev/null +++ b/src/Analyzer/ListNode.h @@ -0,0 +1,48 @@ +#pragma once + +#include + +namespace DB +{ + +/** List node represents list of query tree nodes in query tree. + * + * Example: SELECT column_1, 1, 'constant_value' FROM table. + * column_1, 1, 'constant_value' is list query tree node. + */ +class ListNode; +using ListNodePtr = std::shared_ptr; + +class ListNode final : public IQueryTreeNode +{ +public: + /// Get list nodes + const QueryTreeNodes & getNodes() const + { + return children; + } + + /// Get list nodes + QueryTreeNodes & getNodes() + { + return children; + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::LIST; + } + + void dumpTree(WriteBuffer & buffer, size_t indent) const override; + + String getName() const override; + +protected: + void updateTreeHashImpl(HashState &) const override; + + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; +}; + +} diff --git a/src/Analyzer/MatcherNode.cpp b/src/Analyzer/MatcherNode.cpp new file mode 100644 index 00000000000..4ad38ea6068 --- /dev/null +++ b/src/Analyzer/MatcherNode.cpp @@ -0,0 +1,267 @@ +#include + +#include + +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +const char * toString(MatcherNodeType matcher_node_type) +{ + switch (matcher_node_type) + { + case MatcherNodeType::ASTERISK: + return "ASTERISK"; + case MatcherNodeType::COLUMNS_LIST: + return "COLUMNS_LIST"; + case MatcherNodeType::COLUMNS_REGEXP: + return "COLUMNS_REGEXP"; + } +} + +MatcherNode::MatcherNode(ColumnTransformersNodes column_transformers_) + : MatcherNode(MatcherNodeType::ASTERISK, + {} /*qualified_identifier*/, + {} /*columns_identifiers*/, + {} /*columns_matcher*/, + std::move(column_transformers_) /*column_transformers*/) +{ +} + +MatcherNode::MatcherNode(Identifier qualified_identifier_, ColumnTransformersNodes column_transformers_) + : MatcherNode(MatcherNodeType::ASTERISK, + std::move(qualified_identifier_), + {} /*columns_identifiers*/, + {} /*columns_matcher*/, + std::move(column_transformers_)) +{ +} + +MatcherNode::MatcherNode(std::shared_ptr columns_matcher_, ColumnTransformersNodes column_transformers_) + : MatcherNode(MatcherNodeType::COLUMNS_REGEXP, + {} /*qualified_identifier*/, + {} /*columns_identifiers*/, + std::move(columns_matcher_), + std::move(column_transformers_)) +{ +} + +MatcherNode::MatcherNode(Identifier qualified_identifier_, std::shared_ptr columns_matcher_, ColumnTransformersNodes column_transformers_) + : MatcherNode(MatcherNodeType::COLUMNS_REGEXP, + std::move(qualified_identifier_), + {} /*columns_identifiers*/, + std::move(columns_matcher_), + std::move(column_transformers_)) +{ +} + +MatcherNode::MatcherNode(Identifiers columns_identifiers_, ColumnTransformersNodes column_transformers_) + : MatcherNode(MatcherNodeType::COLUMNS_LIST, + {} /*qualified_identifier*/, + std::move(columns_identifiers_), + {} /*columns_matcher*/, + std::move(column_transformers_)) +{ +} + +MatcherNode::MatcherNode(Identifier qualified_identifier_, Identifiers columns_identifiers_, ColumnTransformersNodes column_transformers_) + : MatcherNode(MatcherNodeType::COLUMNS_LIST, + std::move(qualified_identifier_), + std::move(columns_identifiers_), + {} /*columns_matcher*/, + std::move(column_transformers_)) +{ +} + +MatcherNode::MatcherNode(MatcherNodeType matcher_type_, + Identifier qualified_identifier_, + Identifiers columns_identifiers_, + std::shared_ptr columns_matcher_, + ColumnTransformersNodes column_transformers_) + : matcher_type(matcher_type_) + , qualified_identifier(qualified_identifier_) + , columns_identifiers(columns_identifiers_) + , columns_matcher(columns_matcher_) +{ + auto column_transformers_list_node = std::make_shared(); + + auto & column_transformers_nodes = column_transformers_list_node->getNodes(); + column_transformers_nodes.reserve(column_transformers_.size()); + + for (auto && column_transformer : column_transformers_) + column_transformers_nodes.emplace_back(std::move(column_transformer)); + + children.resize(1); + children[column_transformers_child_index] = std::move(column_transformers_list_node); + + columns_identifiers_set.reserve(columns_identifiers.size()); + + for (auto & column_identifier : columns_identifiers) + columns_identifiers_set.insert(column_identifier.getFullName()); +} + +bool MatcherNode::isMatchingColumn(const std::string & column_name) +{ + if (matcher_type == MatcherNodeType::ASTERISK) + return true; + + if (columns_matcher) + return RE2::PartialMatch(column_name, *columns_matcher); + + return columns_identifiers_set.find(column_name) != columns_identifiers_set.end(); +} + +void MatcherNode::dumpTree(WriteBuffer & buffer, size_t indent) const +{ + buffer << std::string(indent, ' ') << "MATCHER "; + writePointerHex(this, buffer); + + if (!qualified_identifier.empty()) + buffer << ' ' << qualified_identifier.getFullName(); + + buffer << ' ' << toString(matcher_type); + + if (columns_matcher) + { + buffer << ' ' << columns_matcher->pattern(); + } + else if (matcher_type == MatcherNodeType::COLUMNS_LIST) + { + buffer << ' '; + size_t columns_identifiers_size = columns_identifiers.size(); + for (size_t i = 0; i < columns_identifiers_size; ++i) + { + buffer << columns_identifiers[i].getFullName(); + + if (i + 1 != columns_identifiers_size) + buffer << ", "; + } + } + + const auto & column_transformers_list = getColumnTransformers(); + if (!column_transformers_list.getNodes().empty()) + { + buffer << '\n'; + column_transformers_list.dumpTree(buffer, indent + 2); + } +} + +String MatcherNode::getName() const +{ + if (matcher_type == MatcherNodeType::ASTERISK) + return "*"; + + WriteBufferFromOwnString buffer; + buffer << "COLUMNS("; + + if (columns_matcher) + { + buffer << ' ' << columns_matcher->pattern(); + } + else if (matcher_type == MatcherNodeType::COLUMNS_LIST) + { + size_t columns_identifiers_size = columns_identifiers.size(); + for (size_t i = 0; i < columns_identifiers_size; ++i) + { + buffer << columns_identifiers[i].getFullName(); + + if (i + 1 != columns_identifiers_size) + buffer << ", "; + } + } + + buffer << ')'; + return buffer.str(); +} + +void MatcherNode::updateTreeHashImpl(HashState & hash_state) const +{ + hash_state.update(static_cast(matcher_type)); + + const auto & qualified_identifier_full_name = qualified_identifier.getFullName(); + hash_state.update(qualified_identifier_full_name.size()); + hash_state.update(qualified_identifier_full_name); + + for (const auto & identifier : columns_identifiers) + { + const auto & identifier_full_name = identifier.getFullName(); + hash_state.update(identifier_full_name.size()); + hash_state.update(identifier_full_name.data(), identifier_full_name.size()); + } + + if (columns_matcher) + { + const auto & columns_matcher_pattern = columns_matcher->pattern(); + hash_state.update(columns_matcher_pattern.size()); + hash_state.update(columns_matcher_pattern); + } +} + +ASTPtr MatcherNode::toASTImpl() const +{ + ASTPtr result; + + if (matcher_type == MatcherNodeType::ASTERISK) + { + /// For COLUMNS qualified identifier is not supported + if (qualified_identifier.empty()) + { + result = std::make_shared(); + } + else + { + auto qualified_asterisk = std::make_shared(); + auto identifier_parts = qualified_identifier.getParts(); + qualified_asterisk->children.push_back(std::make_shared(std::move(identifier_parts))); + + result = qualified_asterisk; + } + } + else if (columns_matcher) + { + auto regexp_matcher = std::make_shared(); + regexp_matcher->setPattern(columns_matcher->pattern()); + result = regexp_matcher; + } + else + { + auto columns_list_matcher = std::make_shared(); + columns_list_matcher->children.reserve(columns_identifiers.size()); + + for (const auto & identifier : columns_identifiers) + { + auto identifier_parts = identifier.getParts(); + columns_list_matcher->children.push_back(std::make_shared(std::move(identifier_parts))); + } + + result = columns_list_matcher; + } + + for (const auto & child : children) + result->children.push_back(child->toAST()); + + return result; +} + +QueryTreeNodePtr MatcherNode::cloneImpl() const +{ + MatcherNodePtr matcher_node = std::make_shared(); + + matcher_node->matcher_type = matcher_type; + matcher_node->qualified_identifier = qualified_identifier; + matcher_node->columns_identifiers = columns_identifiers; + matcher_node->columns_matcher = columns_matcher; + matcher_node->columns_identifiers_set = columns_identifiers_set; + + return matcher_node; +} + +} diff --git a/src/Analyzer/MatcherNode.h b/src/Analyzer/MatcherNode.h new file mode 100644 index 00000000000..924c028f8f9 --- /dev/null +++ b/src/Analyzer/MatcherNode.h @@ -0,0 +1,171 @@ +#pragma once + +#include + +#include +#include +#include +#include + + +namespace DB +{ + +/** Matcher query tree node. + * Matcher can be unqualified with identifier and qualified with identifier. + * It can be asterisk or COLUMNS('regexp') or COLUMNS(column_name_1, ...). + * In result we have 6 possible options: + * Unqualified + * 1. * + * 2. COLUMNS('regexp') + * 3. COLUMNS(column_name_1, ...) + * + * Qualified: + * 1. identifier.* + * 2. identifier.COLUMNS('regexp') + * 3. identifier.COLUMNS(column_name_1, ...) + * + * The main difference between matcher and identifier is that matcher cannot have alias. + * This simplifies analysis for matchers. + * + * How to resolve matcher during query analysis pass: + * 1. If matcher is unqualified, we use tables of current scope and try to resolve matcher from it. + * 2. If matcher is qualified: + * First try to resolve identifier part as query expression. + * Try expressions from aliases, then from tables (can be changed using prefer_column_name_to_alias setting). + * If identifier is resolved as expression. If expression is compound apply matcher to it, otherwise throw exception. + * Example: SELECT compound_column AS a, a.* FROM test_table. + * Example: SELECT compound_column.* FROM test_table. + * + * If identifier is not resolved as expression try to resolve it as table. + * If identifier is resolved as table then apply matcher to it. + * Example: SELECT test_table.* FROM test_table. + * Example: SELECT a.* FROM test_table AS a. + * + * Additionaly each matcher can contain transformers, check ColumnTransformers.h. + * In query tree matchers column transformers are represended as ListNode. + */ +enum class MatcherNodeType +{ + ASTERISK, + COLUMNS_REGEXP, + COLUMNS_LIST +}; + +const char * toString(MatcherNodeType matcher_node_type); + +class MatcherNode; +using MatcherNodePtr = std::shared_ptr; + +class MatcherNode final : public IQueryTreeNode +{ +public: + /// Variant unqualified asterisk + explicit MatcherNode(ColumnTransformersNodes column_transformers_ = {}); + + /// Variant qualified asterisk + explicit MatcherNode(Identifier qualified_identifier_, ColumnTransformersNodes column_transformers_ = {}); + + /// Variant unqualified COLUMNS('regexp') + explicit MatcherNode(std::shared_ptr columns_matcher_, ColumnTransformersNodes column_transformers_ = {}); + + /// Variant qualified COLUMNS('regexp') + explicit MatcherNode(Identifier qualified_identifier_, std::shared_ptr columns_matcher_, ColumnTransformersNodes column_transformers_ = {}); + + /// Variant unqualified COLUMNS(column_name_1, ...) + explicit MatcherNode(Identifiers columns_identifiers_, ColumnTransformersNodes column_transformers_ = {}); + + /// Variant qualified COLUMNS(column_name_1, ...) + explicit MatcherNode(Identifier qualified_identifier_, Identifiers columns_identifiers_, ColumnTransformersNodes column_transformers_ = {}); + + /// Get matcher type + MatcherNodeType getMatcherType() const + { + return matcher_type; + } + + /// Is this matcher represented by asterisk + bool isAsteriskMatcher() const + { + return matcher_type == MatcherNodeType::ASTERISK; + } + + /// Is this matcher represented by COLUMNS + bool isColumnsMatcher() const + { + return matcher_type == MatcherNodeType::COLUMNS_REGEXP || matcher_type == MatcherNodeType::COLUMNS_LIST; + } + + /// Returns true if matcher qualified with identifier, false otherwise + bool isQualified() const + { + return !qualified_identifier.empty(); + } + + /// Get qualified identifier + const Identifier & getQualifiedIdentifier() const + { + return qualified_identifier; + } + + /// Get columns matcher. Valid only if this matcher has type COLUMNS_REGEXP. + const std::shared_ptr & getColumnsMatcher() const + { + return columns_matcher; + } + + /// Get columns matcher. Valid only if this matcher has type COLUMNS_LIST. + const Identifiers & getColumnsIdentifiers() const + { + return columns_identifiers; + } + + /** Get column transformers + * Client can expect that node in this list is subclass of IColumnTransformerNode. + */ + const ListNode & getColumnTransformers() const + { + return children[column_transformers_child_index]->as(); + } + + const QueryTreeNodePtr & getColumnTransformersNode() const + { + return children[column_transformers_child_index]; + } + + /// Returns true if matcher match column name, false otherwise + bool isMatchingColumn(const std::string & column_name); + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::ASTERISK; + } + + void dumpTree(WriteBuffer & buffer, size_t indent) const override; + + String getName() const override; + +protected: + void updateTreeHashImpl(HashState & hash_state) const override; + + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + explicit MatcherNode(MatcherNodeType matcher_type_, + Identifier qualified_identifier_, + Identifiers columns_identifiers_, + std::shared_ptr columns_matcher_, + ColumnTransformersNodes column_transformers_); + + MatcherNodeType matcher_type; + Identifier qualified_identifier; + Identifiers columns_identifiers; + std::shared_ptr columns_matcher; + std::unordered_set columns_identifiers_set; + + static constexpr size_t column_transformers_child_index = 0; +}; + +} diff --git a/src/Analyzer/MultiIfToIfPass.cpp b/src/Analyzer/MultiIfToIfPass.cpp new file mode 100644 index 00000000000..08dd64fa326 --- /dev/null +++ b/src/Analyzer/MultiIfToIfPass.cpp @@ -0,0 +1,49 @@ +#include + +#include +#include +#include + +namespace DB +{ + +namespace +{ + class MultiIfToIfVisitorMatcher + { + public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + FunctionOverloadResolverPtr if_function_overload_resolver; + }; + + static void visit(QueryTreeNodePtr & node, Data & data) + { + auto * function_node = node->as(); + if (!function_node || function_node->getFunctionName() != "multiIf") + return; + + if (function_node->getArguments().getNodes().size() != 3) + return; + + auto result_type = function_node->getResultType(); + function_node->resolveAsFunction(data.if_function_overload_resolver, result_type); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) + { + return true; + } + }; +} + +void MultiIfToIfPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + MultiIfToIfVisitorMatcher::Data data{FunctionFactory::instance().get("if", context)}; + MultiIfToIfVisitorMatcher::Visitor visitor(data); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/MultiIfToIfPass.h b/src/Analyzer/MultiIfToIfPass.h new file mode 100644 index 00000000000..fc4859d1769 --- /dev/null +++ b/src/Analyzer/MultiIfToIfPass.h @@ -0,0 +1,25 @@ +#pragma once + +#include + +namespace DB +{ + +/** Convert multiIf with single argument into if. + * Example: SELECT multiIf(x, 1, 0); + * Result: SELECT if(x, 1, 0); + */ +class MultiIfToIfPass : public IQueryTreePass +{ +public: + + String getName() override { return "MultiIfToIf"; } + + String getDescription() override { return "Optimize multiIf to if for single argument"; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} + diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp new file mode 100644 index 00000000000..56c43c7223d --- /dev/null +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -0,0 +1,2024 @@ +#include + +#include + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include + +#include + +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; + extern const int UNKNOWN_IDENTIFIER; + extern const int LOGICAL_ERROR; + extern const int CYCLIC_ALIASES; + extern const int INCORRECT_RESULT_OF_SCALAR_SUBQUERY; + extern const int BAD_ARGUMENTS; +} + +/** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. + * And additional documentation is writter for each method, where special cases are described in detail. + * + * Each node in query must be resolved. For each query tree node resolved state is specific. + * + * For constant node no resolve process exists, it is resolved during construction. + * + * For table node no resolve process exists, it is resolved during construction. + * + * For function node to be resolved parameters and arguments must be resolved, must be initialized with concrete aggregate or + * non aggregate function and with result type. + * + * For lambda node there can be 2 different cases. + * 1. Standalone: WITH (x -> x + 1) AS lambda SELECT lambda(1); Such lambdas are inlined in query tree during query analysis pass. + * 2. Function arguments: WITH (x -> x + 1) AS lambda SELECT arrayMap(lambda, [1, 2, 3]); For such lambda resolution must + * set concrete lambda arguments (initially they are identifier nodes) and resolve lambda expression body. + * + * For query node resolve process must resolve all its inner nodes. + * + * For matcher node resolve process must replace it with matched nodes. + * + * For identifier node resolve process must replace it with concrete non identifier node. This part is most complex because + * for identifier resolution scopes and identifier lookup context play important part. + * + * ClickHouse SQL support lexical scoping for identifier resolution. Scope can be defined by query node or by expression node. + * Expression nodes that can define scope are lambdas and table ALIAS columns. + * + * Identifier lookup context can be expression, function, table. + * + * Examples: WITH (x -> x + 1) as func SELECT func() FROM func; During function `func` resolution identifier lookup is performed + * in function context. + * + * If there are no information of identifier context rules are following: + * 1. Try to resolve identifier in expression context. + * 2. Try to resolve identifier in function context, if it is allowed. Example: SELECT func; Here func identifier cannot be resolved in function context + * because query projection does not support that. + * 3. Try to resolve identifier in talbe context, if it is allowed. Example: SELECT table; Here table identifier cannot be resolved in function context + * because query projection does not support that. + * + * TODO: This does not supported properly before, because matchers could not be resolved from aliases. + * + * Identifiers are resolved with following resules: + * Resolution starts with current scope. + * 1. Try to resolve identifier from expression scope arguments. Lambda expression arguments are greatest priority. + * 2. Try to resolve identifier from aliases. + * 3. Try to resolve identifier from tables if scope is query. + * Steps 2 and 3 can be changed using prefer_column_name_to_alias setting. + * If identifier could not be resolved in current scope, resolution must be continued in parent scopes. + * 4. Try to resolve identifier from parent scopes. + * + * Additional rules about aliases and scopes. + * 1. Parent scope cannot refer alias from child scope. + * 2. Child scope can refer to alias in parent scope. + * + * Example: SELECT arrayMap(x -> x + 1 AS a, [1,2,3]), a; Identifier a is unknown in parent scope. + * Example: SELECT a FROM (SELECT 1 as a); Here we do not refer to alias a from child query scope. But we query it projection result, similar to tables. + * Example: WITH 1 as a SELECT (SELECT a) as b; Here in child scope identifier a is resolved using alias from parent scope. + * + * Additional rules about identifier binding. + * Bind for identifier to entity means that identifier first part match some node during analysis. + * If other parts of identifier cannot be resolved in that node, exception must be throwed. + * + * Example: + * CREATE TABLE test_table (id UInt64, compound_value Tuple(value UInt64)) ENGINE=TinyLog; + * SELECT compound_value.value, 1 AS compound_value FROM test_table; + * Identifier first part compound_value bound to entity with alias compound_value, but nested identifier part cannot be resolved from entity, + * lookup should not be continued, and exception must be throwed because if lookup continues that way identifier can be resolved from tables. + * + * TODO: This was not supported properly before analyzer because nested identifier could not be resolved from alias. + * + * More complex example: + * CREATE TABLE test_table (id UInt64, value UInt64) ENGINE=TinyLog; + * WITH cast(('Value'), 'Tuple (value UInt64') AS value SELECT (SELECT value FROM test_table); + * Identifier first part value bound to test_table column value, but nested identifier part cannot be resolved from it, + * lookup should not be continued, and exception must be throwed because if lookup continues identifier can be resolved from parent scope. + * + * TODO: Add expression name into query tree node. Example: SELECT plus(1, 1). Result: SELECT 2. Expression name of constant node should be 2. + * TODO: Table identifiers with optional UUID. + * TODO: Support STRICT except, replace matchers. + * TODO: Support multiple entities with same alias. + * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); + * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. + * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. + * TODO: CTE, JOIN, ARRAY JOIN, bulding sets, grouping, in. + */ + +/// Identifier lookup context +enum class IdentifierLookupContext : uint8_t +{ + EXPRESSION = 0, + FUNCTION, + TABLE, +}; + +static const char * toString(IdentifierLookupContext identifier_lookup_context) +{ + switch (identifier_lookup_context) + { + case IdentifierLookupContext::EXPRESSION: return "EXPRESSION"; + case IdentifierLookupContext::FUNCTION: return "FUNCTION"; + case IdentifierLookupContext::TABLE: return "TABLE"; + } +} + +static const char * toStringLowercase(IdentifierLookupContext identifier_lookup_context) +{ + switch (identifier_lookup_context) + { + case IdentifierLookupContext::EXPRESSION: return "expression"; + case IdentifierLookupContext::FUNCTION: return "function"; + case IdentifierLookupContext::TABLE: return "table"; + } +} + +/** Structure that represent identifier lookup during query analysis. + * Lookup can be in query expression, function, table context. + */ +struct IdentifierLookup +{ + Identifier identifier; + IdentifierLookupContext lookup_context; + + bool isExpressionLookup() const + { + return lookup_context == IdentifierLookupContext::EXPRESSION; + } + + bool isFunctionLookup() const + { + return lookup_context == IdentifierLookupContext::FUNCTION; + } + + bool isTableLookup() const + { + return lookup_context == IdentifierLookupContext::TABLE; + } + + String dump() const + { + return identifier.getFullName() + ' ' + toString(lookup_context); + } +}; + +inline bool operator==(const IdentifierLookup & lhs, const IdentifierLookup & rhs) +{ + return lhs.identifier.getFullName() == rhs.identifier.getFullName() && lhs.lookup_context == rhs.lookup_context; +} + +inline bool operator!=(const IdentifierLookup & lhs, const IdentifierLookup & rhs) +{ + return !(lhs == rhs); +} + +struct IdentifierLookupHash +{ + size_t operator()(const IdentifierLookup & identifier_lookup) const + { + return std::hash()(identifier_lookup.identifier.getFullName()) ^ static_cast(identifier_lookup.lookup_context); + } +}; + +struct StorageIDHash +{ + size_t operator()(const StorageID & storage_id) const + { + return std::hash()(storage_id.getFullNameNotQuoted()); + } +}; + +struct StorageColumns +{ + NamesAndTypesList column_names_and_types; + std::unordered_set column_identifier_first_parts; + + bool canBindIdentifier(IdentifierView identifier) + { + return column_identifier_first_parts.find(std::string(identifier.at(0))) != column_identifier_first_parts.end(); + } +}; + +class ExpressionsStack +{ +public: + void pushNode(const QueryTreeNodePtr & node) + { + if (node->hasAlias()) + { + expressions.emplace_back(node.get(), node->getAlias()); + + auto [it, inserted] = expressions_aliases.emplace(expressions.back().second); + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expression with alias {} already exists in stack", + node->getAlias()); + return; + } + + expressions.emplace_back(node.get(), std::string()); + } + + void popNode() + { + const auto & [_, top_expression_alias] = expressions.back(); + if (!top_expression_alias.empty()) + expressions_aliases.erase(top_expression_alias); + + expressions.pop_back(); + } + + const IQueryTreeNode * getRoot() const + { + if (expressions.empty()) + return nullptr; + + return expressions.front().first; + } + + const IQueryTreeNode * getTop() const + { + if (expressions.empty()) + return nullptr; + + return expressions.back().first; + } + + bool hasExpressionWithAlias(const std::string & alias) const + { + return expressions_aliases.find(alias) != expressions_aliases.end(); + } + + size_t size() const + { + return expressions.size(); + } + + bool empty() const + { + return expressions.empty(); + } + + void dump(WriteBuffer & buffer) + { + buffer << expressions.size() << '\n'; + + for (auto & [expression, alias] : expressions) + { + buffer << "Expression "; + buffer << expression->formatASTForErrorMessage(); + + if (!alias.empty()) + buffer << " alias " << alias; + + buffer << '\n'; + } + } + +private: + std::vector> expressions; + std::unordered_set expressions_aliases; +}; + +struct IdentifierResolveScope +{ + /// Construct identifier resolve scope using scope node, and parent scope + IdentifierResolveScope(QueryTreeNodePtr scope_node_, IdentifierResolveScope * parent_scope_) + : scope_node(std::move(scope_node_)) + , parent_scope(parent_scope_) + {} + + QueryTreeNodePtr scope_node; + IdentifierResolveScope * parent_scope = nullptr; + + std::unordered_map identifier_lookup_to_node; + + /// Lambda argument can be expression like constant, column, or it can be function + std::unordered_map expression_argument_name_to_node; + + /// Alias name to query expression node + std::unordered_map alias_name_to_expression_node; + + /// Alias name to lambda node + std::unordered_map alias_name_to_lambda_node; + + /// Alias name to table expression node + std::unordered_map alias_name_to_table_expression_node; + + /// Current scope expression in resolve process stack + ExpressionsStack expressions_in_resolve_process_stack; + + /// Current scope expression + std::unordered_set non_cached_identifier_lookups_during_expression_resolve; + + /// Allow to check parent scopes if identifier cannot be resolved in current scope + bool allow_to_check_parent_scopes = true; + + /// Dump identifier resolve scope + void dump(WriteBuffer & buffer) + { + buffer << "Scope node " << scope_node->formatASTForErrorMessage() << '\n'; + buffer << "Identifier lookup to node " << identifier_lookup_to_node.size() << '\n'; + for (const auto & [identifier, node] : identifier_lookup_to_node) + buffer << "Identifier " << identifier.dump() << " node " << node->formatASTForErrorMessage() << '\n'; + + buffer << "Expression argument name to node " << expression_argument_name_to_node.size() << '\n'; + for (const auto & [alias_name, node] : expression_argument_name_to_node) + buffer << "Alias name " << alias_name << " node " << node->formatASTForErrorMessage() << '\n'; + + buffer << "Alias name to expression node " << alias_name_to_expression_node.size() << '\n'; + for (const auto & [alias_name, node] : alias_name_to_expression_node) + buffer << "Alias name " << alias_name << " expression node " << node->formatASTForErrorMessage() << '\n'; + + buffer << "Alias name to function node " << alias_name_to_lambda_node.size() << '\n'; + for (const auto & [alias_name, node] : alias_name_to_lambda_node) + buffer << "Alias name " << alias_name << " lambda node " << node->formatASTForErrorMessage() << '\n'; + + buffer << "Alias name to table expression node " << alias_name_to_table_expression_node.size() << '\n'; + for (const auto & [alias_name, node] : alias_name_to_table_expression_node) + buffer << "Alias name " << alias_name << " table node " << node->formatASTForErrorMessage() << '\n'; + + buffer << "Expression resolve process stack " << '\n'; + expressions_in_resolve_process_stack.dump(buffer); + + buffer << "Allow to check parent scopes " << allow_to_check_parent_scopes << '\n'; + // buffer << "Parent scope " << parent_scope << '\n'; + } +}; + +class QueryAnalyzer +{ +public: + explicit QueryAnalyzer(ContextPtr context_) + : context(std::move(context_)) + {} + + void resolve(QueryTreeNodePtr node) + { + IdentifierResolveScope scope(node, nullptr /*parent_scope*/); + + if (node->getNodeType() == QueryTreeNodeType::QUERY) + { + resolveQuery(node, scope); + } + else if (node->getNodeType() == QueryTreeNodeType::LIST) + { + resolveExpressionNodeList(node, scope, false /*allow_lambda_expression*/); + } + else if (node->getNodeType() == QueryTreeNodeType::FUNCTION) + { + resolveExpressionNode(node, scope, false /*allow_lambda_expression*/); + } + else if (node->getNodeType() == QueryTreeNodeType::LAMBDA) + { + resolveLambda(node, {}, scope); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Node {} with type {} is not supported by query analyzer. Supported nodes are query, list, function, lambda.", + node->formatASTForErrorMessage(), + node->getNodeTypeName()); + } + } + +private: + /// Utility functions + + static bool isNodePartOfSubtree(const IQueryTreeNode * node, const IQueryTreeNode * root); + + static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path); + + static QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); + + void evaluateScalarSubquery(QueryTreeNodePtr & query_tree_node); + + /// Resolve identifier functions + + QueryTreeNodePtr tryResolveIdentifierFromExpressionArguments(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); + + QueryTreeNodePtr tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); + + QueryTreeNodePtr tryResolveIdentifierFromTables(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); + + QueryTreeNodePtr tryResolveIdentifierInCurrentScope(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & current_scope); + + QueryTreeNodePtr tryResolveIdentifierInParentScopes(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); + + QueryTreeNodePtr tryResolveIdentifier(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); + + /// Resolve query tree nodes functions + + QueryTreeNodePtr resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); + + void resolveLambda(QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope); + + void resolveFunction(QueryTreeNodePtr & function_node, IdentifierResolveScope & scope); + + void resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression); + + void resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression); + + void resolveQuery(QueryTreeNodePtr & query_tree_node, IdentifierResolveScope & scope); + + /// Query analyzer context + ContextPtr context; + + /// Lambdas that are currently in resolve process + std::unordered_set lambdas_in_resolve_process; + + /// Storage id to storage columns cache + std::unordered_map storage_id_to_columns_cache; +}; + +/// Utility functions implementation + +bool QueryAnalyzer::isNodePartOfSubtree(const IQueryTreeNode * node, const IQueryTreeNode * root) +{ + std::vector nodes_to_process; + nodes_to_process.push_back(root); + + while (!nodes_to_process.empty()) + { + const auto * subtree_node = nodes_to_process.back(); + nodes_to_process.pop_back(); + + if (subtree_node == node) + return true; + + for (const auto & child : subtree_node->getChildren()) + { + if (child) + nodes_to_process.push_back(child.get()); + } + } + + return false; +} + +/** Wrap expression node in tuple element function calls for nested paths. + * Example: Expression node: compound_expression. Nested path: nested_path_1.nested_path_2. + * Result: tupleElement(tupleElement(compound_expression, 'nested_path_1'), 'nested_path_2'). + */ +QueryTreeNodePtr QueryAnalyzer::wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path) +{ + size_t nested_path_parts_size = nested_path.getPartsSize(); + for (size_t i = 0; i < nested_path_parts_size; ++i) + { + const auto & nested_path_part = nested_path[i]; + auto tuple_element_function = std::make_shared("tupleElement"); + tuple_element_function->getArguments().getNodes().push_back(expression_node); + tuple_element_function->getArguments().getNodes().push_back(std::make_shared(nested_path_part)); + expression_node = tuple_element_function; + } + + return expression_node; +} + +/** Try to get lambda node from sql user defined functions if sql user defined function with function name exists. + * Returns lambda node if function exists, nullptr otherwise. + */ +QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name, const ContextPtr & context) +{ + auto user_defined_function = UserDefinedSQLFunctionFactory::instance().tryGet(function_name); + if (!user_defined_function) + return {}; + + const auto & create_function_query = user_defined_function->as(); + auto result_node = buildQueryTree(create_function_query->function_core, context); + if (result_node->getNodeType() != QueryTreeNodeType::LAMBDA) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "SQL user defined function {} must represent lambda expression. Actual {}", + function_name, + create_function_query->function_core->formatForErrorMessage()); + + return result_node; +} + +/** Evaluate scalar subquery. + * In result of this function node will be replaced by constant node. + */ +void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node) +{ + auto subquery_context = Context::createCopy(context); + Settings subquery_settings = context->getSettings(); + subquery_settings.max_result_rows = 1; + subquery_settings.extremes = false; + subquery_context->setSettings(subquery_settings); + + if (!subquery_context->hasQueryContext()) + { + // auto subquery_query_context = subquery_context->getQueryContext(); + // for (const auto & it : data.scalars) + // context->addScalar(it.first, it.second); + } + + size_t subquery_depth = 0; + auto options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth + 1, true); + + auto interpreter = std::make_unique(node->toAST(), options, subquery_context); + + auto io = interpreter->execute(); + + Block block; + PullingAsyncPipelineExecutor executor(io.pipeline); + io.pipeline.setProgressCallback(context->getProgressCallback()); + while (block.rows() == 0 && executor.pull(block)) + { + } + + if (block.rows() == 0) + { + auto types = interpreter->getSampleBlock().getDataTypes(); + if (types.size() != 1) + types = {std::make_shared(types)}; + + auto & type = types[0]; + if (!type->isNullable()) + { + if (!type->canBeInsideNullable()) + throw Exception(ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY, + "Scalar subquery returned empty result of type {} which cannot be Nullable.", + type->getName()); + + type = makeNullable(type); + } + + node = std::make_shared(Null()); + return; + } + + if (block.rows() != 1) + throw Exception(ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY, "Scalar subquery returned more than one row"); + + Block tmp_block; + while (tmp_block.rows() == 0 && executor.pull(tmp_block)) + { + } + + if (tmp_block.rows() != 0) + throw Exception(ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY, "Scalar subquery returned more than one row"); + + block = materializeBlock(block); + size_t columns = block.columns(); + + // Block scalar; + Field scalar_value; + DataTypePtr scalar_type; + + if (columns == 1) + { + auto & column = block.getByPosition(0); + /// Here we wrap type to nullable if we can. + /// It is needed cause if subquery return no rows, it's result will be Null. + /// In case of many columns, do not check it cause tuple can't be nullable. + if (!column.type->isNullable() && column.type->canBeInsideNullable()) + { + column.type = makeNullable(column.type); + column.column = makeNullable(column.column); + } + + column.column->get(0, scalar_value); + scalar_type = column.type; + } + else + { + auto tuple_column = ColumnTuple::create(block.getColumns()); + tuple_column->get(0, scalar_value); + scalar_type = std::make_shared(block.getDataTypes()); + } + + node = std::make_shared(std::move(scalar_value), std::move(scalar_type)); +} + +/// Resolve identifier functions implementation + +/** Resolve identifier from expression arguments. + * + * Expression arguments can be initialized during lambda analysis or they could be provided externally. + * Expression arguments must be already resolved nodes. This is client responsibility to resolve them during scope initialization. + * + * Resolve strategy: + * 1. Try to bind identifier to scope argument name to node map. + * 2. If identifier is binded but expression context and node type are incompatible return nullptr. + * + * It is important to support edge cases, where we lookup for table or function node, but argument has same name. + * Example: WITH (x -> x + 1) AS func, (func -> func(1) + func) AS lambda SELECT lambda(1); + * + * 3. If identifier is compound and identifier lookup is in expression context, pop first part from identifier lookup and wrap node + * using nested parts of identifier using `wrapExpressionNodeInTupleElement` function. + */ +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +{ + const auto & identifier_bind_part = identifier_lookup.identifier.front(); + + auto it = scope.expression_argument_name_to_node.find(identifier_bind_part); + if (it == scope.expression_argument_name_to_node.end()) + return {}; + + if (identifier_lookup.isExpressionLookup() && it->second->getNodeType() == QueryTreeNodeType::LAMBDA) + return {}; + else if (identifier_lookup.isTableLookup() && it->second->getNodeType() != QueryTreeNodeType::TABLE) + return {}; + + if (identifier_lookup.identifier.isCompound() && identifier_lookup.isExpressionLookup()) + { + auto nested_path = IdentifierView(identifier_lookup.identifier); + nested_path.popFirst(); + + auto tuple_element_result = wrapExpressionNodeInTupleElement(it->second, nested_path); + resolveFunction(tuple_element_result, scope); + + return tuple_element_result; + } + + return it->second; +} + +/** Visitor that extracts expression and function aliases from node and initialize scope tables with it. + * Does not go into child lambdas and queries. + * If there are multiple entities with same alias, exception is raised. + * + * TODO: Maybe better for this visitor to handle QueryNode. Handle table nodes. + * + * Important: + * Identifier nodes with aliases are added both in alias to expression and alias to function map. + * + * These is necessary because identifier with alias can give any node alias name, expression or function. + * + * TODO: Disable identifier with alias node propagation for table nodes. This can occur only for special functions + * if their argument can be table. + * + * Example: + * WITH (x -> x + 1) AS id, id AS value SELECT value(1); + * In this example id as value is identifier node that has alias, during scope initialization we cannot derive + * that id is actually lambda or expression. + * + * There are no easy solution here, without trying to make full featured expression resolution at this stage, because example can be much complex: + * Example: + * WITH (x -> x + 1) AS id, id AS id_1, id_1 AS id_2 SELECT id_2(1); + * + * It is client responsibility after resolving identifier node with alias, make following actions: + * 1. If identifier node was resolved in function scope, remove alias from scope expression map. + * 2. If identifier node was resolved in expression scope, remove alias from scope function map. + * + * That way we separate alias map initialization and expressions resolution. + */ +class ScopeAliasVisitorMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + IdentifierResolveScope & scope; + }; + + static void visit(QueryTreeNodePtr & node, Data & data) + { + updateAliasesIfNeeded(data, node, false); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child, Data & data) + { + if (auto * lambda_node = child->as()) + { + updateAliasesIfNeeded(data, child, true); + return false; + } + else if (auto * query_tree_node = child->as()) + { + updateAliasesIfNeeded(data, child, false); + return false; + } + + return !(child->as()); + } +private: + static void updateAliasesIfNeeded(Data & data, const QueryTreeNodePtr & node, bool function_node) + { + if (!node->hasAlias()) + return; + + const auto & alias = node->getAlias(); + auto throw_exception = [&]() + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Having multiple alises with same name {} is not allowed. In scope {}", + alias, + data.scope.scope_node->formatASTForErrorMessage()); + }; + + if (function_node) + { + if (data.scope.alias_name_to_expression_node.contains(alias)) + throw_exception(); + + auto [_, inserted] = data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); + if (!inserted) + throw_exception(); + + return; + } + + if (data.scope.alias_name_to_lambda_node.contains(alias)) + throw_exception(); + + auto [_, inserted] = data.scope.alias_name_to_expression_node.insert(std::make_pair(alias, node)); + if (!inserted) + throw_exception(); + + /// If node is identifier put it also in scope alias name to lambda node map + if (node->getNodeType() == QueryTreeNodeType::IDENTIFIER) + data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); + } +}; + +using ScopeAliasVisitor = ScopeAliasVisitorMatcher::Visitor; + +/** Resolve identifier from scope aliases. + * + * Resolve strategy: + * 1. If alias is registered current expressions that are in resolve process and if last expression is not part of first expression subtree + * throw cyclic aliases exception. + * Otherwise prevent cache usage for identifier lookup and return nullptr. + * + * This is special scenario where identifier has name the same as alias name in one of its parent expressions including itself. + * In such case we cannot resolve identifier from aliases because of recursion. It is client responsibility to register and deregister alias + * names during expressions resolve. + * + * We must prevent cache usage for lookup because lookup outside of expression is supposed to return other value. + * Example: SELECT (id + 1) AS id, id + 2. Lookup for id inside (id + 1) as id should return id from table, but lookup (id + 2) should return + * (id + 1) AS id. + * + * Below cases should work: + * Example: + * SELECT id AS id FROM test_table; + * SELECT value.value1 AS value FROM test_table; + * SELECT (id + 1) AS id FROM test_table; + * SELECT (1 + (1 + id)) AS id FROM test_table; + * + * Below cases should throw cyclic aliases exception: + * SELECT (id + b) AS id, id as b FROM test_table; + * SELECT (1 + b + 1 + id) AS id, b as c, id as b FROM test_table; + * + * 2. Depending on IdentifierLookupContext get alias name to node map from IdentifierResolveScope. + * 3. Try to bind identifier to alias name in map. If there are no such binding return nullptr. + * 4. Add node into current expressions to resolve. TODO: Handle lambdas and tables properly. + * + * 5. If node in map is not resolved, resolve it. It is important because for result type of identifier lookup node can depend on it. + * Example: SELECT value.a, cast('(1)', 'Tuple(a UInt64)') AS value; + * + * Special case for IdentifierNode, if node is identifier depending on lookup context we need to erase entry from expression or lambda map. + * Check ScopeAliasVisitorMatcher documentation. + * + * Special case for QueryNode, if lookup context is expression, evaluate it as scalar subquery. + * + * 6. If identifier is compound and identifier lookup is in expression context, pop first part from identifier lookup and wrap alias node + * using nested parts of identifier using `wrapExpressionNodeInTupleElement` function. + * + * Example: SELECT value AS alias, alias.nested_path. + * Result: SELECT value AS alias, tupleElement(value, 'nested_path') value.nested_path. + * + * 7. If identifier lookup is in expression context, clone result expression. + * 8. Pop node from current expressions to resolve. + */ +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +{ + const auto & identifier_bind_part = identifier_lookup.identifier.front(); + auto get_alias_name_to_node_map = [&]() -> std::unordered_map & + { + if (identifier_lookup.isExpressionLookup()) + return scope.alias_name_to_expression_node; + else if (identifier_lookup.isFunctionLookup()) + return scope.alias_name_to_lambda_node; + + return scope.alias_name_to_table_expression_node; + }; + + auto & alias_name_to_node_map = get_alias_name_to_node_map(); + auto it = alias_name_to_node_map.find(identifier_bind_part); + + if (it == alias_name_to_node_map.end()) + return {}; + + if (!it->second) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Node with alias {} is not valid. In scope {}", + identifier_bind_part, + scope.scope_node->formatASTForErrorMessage()); + + if (scope.expressions_in_resolve_process_stack.hasExpressionWithAlias(identifier_bind_part)) + { + const auto * root_expression = scope.expressions_in_resolve_process_stack.getRoot(); + const auto * top_expression = scope.expressions_in_resolve_process_stack.getTop(); + + if (!isNodePartOfSubtree(top_expression, root_expression)) + { + throw Exception(ErrorCodes::CYCLIC_ALIASES, + "Cyclic aliases for identifier {}. In scope {}", + identifier_lookup.identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + } + + scope.non_cached_identifier_lookups_during_expression_resolve.insert(identifier_lookup); + return {}; + } + + scope.expressions_in_resolve_process_stack.pushNode(it->second); + + /// Resolve expression if necessary + if (auto * alias_identifier = it->second->as()) + { + it->second = tryResolveIdentifier(IdentifierLookup{alias_identifier->getIdentifier(), identifier_lookup.lookup_context}, scope); + + if (!it->second) + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Unknown {} identifier {}. In scope {}", + toStringLowercase(identifier_lookup.lookup_context), + identifier_lookup.identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + + /** During collection of aliases if node is identifier and has alias, we cannot say if it is + * column or function node. Check ScopeAliasVisitor documentation for clarification. + * + * If we resolved identifier node as expression, we must remove identifier node alias from + * function alias map. + * If we resolved identifier node as function, we must remove identifier node alias from + * expression alias map. + */ + if (identifier_lookup.isExpressionLookup()) + scope.alias_name_to_lambda_node.erase(identifier_bind_part); + else if (identifier_lookup.isFunctionLookup() && it->second) + scope.alias_name_to_expression_node.erase(identifier_bind_part); + } + else if (auto * function = it->second->as()) + { + resolveFunction(it->second, scope); + } + else if (auto * query = it->second->as()) + { + IdentifierResolveScope subquery_scope(it->second, &scope /*parent_scope*/); + resolveQuery(it->second, subquery_scope); + + if (identifier_lookup.isExpressionLookup()) + evaluateScalarSubquery(it->second); + } + + if (!it->second) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Node with alias {} is not valid. In scope {}", + identifier_bind_part, + scope.scope_node->formatASTForErrorMessage()); + + QueryTreeNodePtr result = it->second; + + if (identifier_lookup.identifier.isCompound() && identifier_lookup.isExpressionLookup()) + { + auto nested_path = IdentifierView(identifier_lookup.identifier); + nested_path.popFirst(); + + auto tuple_element_result = wrapExpressionNodeInTupleElement(it->second, nested_path); + resolveFunction(tuple_element_result, scope); + + result = tuple_element_result; + } + else if (identifier_lookup.isExpressionLookup()) + { + /** If expression node was resolved throught aliases we must clone it to keep query tree state valid. + * + * Example: + * If we have query SELECT 1 as a, a + * If we do not clone node, after query analysis query will look like SELECT 1 as a, 1 as a. + * + * This query is broken because multiple aliases with same name are not allowed in query tree. + */ + result = it->second->clone(); + result->removeAlias(); + } + + scope.expressions_in_resolve_process_stack.popNode(); + + return result; +} + +/** Resolve identifier from scope tables. + * + * 1. If there is no table node in scope, or identifier is in function lookup context return nullptr. + * 2. If identifier is in table lookup context, check if it has 1 or 2 parts, otherwise throw exception. + * If identifer has 2 parts try to match it with database_name and table_name. + * If identifier has 1 part try to match it with table_name, then try to match it with table alias. + * 3. If identifier is in expression lookup context, we first need to bind identifier to some table column using identifier first part. + * Start with identifier first part, if it match some column name in table try to get column with full identifier name. + * TODO: Need to check if it is okay to throw exception if compound identifier first part bind to column but column is not valid. + */ +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +{ + if (identifier_lookup.isFunctionLookup()) + return {}; + + auto * query_scope_node = scope.scope_node->as(); + if (!query_scope_node || !query_scope_node->getFrom()) + return {}; + + auto from_node = query_scope_node->getFrom(); + auto * table_node = query_scope_node->getFrom()->as(); + + if (!table_node) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "From does not contain table node"); + + const auto & identifier = identifier_lookup.identifier; + const auto & path_start = identifier.getParts().front(); + auto storage_id = table_node->getStorageID(); + auto & table_name = storage_id.table_name; + auto & database_name = storage_id.database_name; + + if (identifier_lookup.isTableLookup()) + { + size_t parts_size = identifier_lookup.identifier.getPartsSize(); + if (parts_size != 1 && parts_size != 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Expected identifier {} to contain 1 or 2 parts size to be resolved as table. In scope {}", + identifier_lookup.identifier.getFullName(), + table_node->formatASTForErrorMessage()); + + if (parts_size == 1 && path_start == table_name) + return from_node; + else if (parts_size == 2 && path_start == database_name && identifier[1] == table_name) + return from_node; + else + return {}; + } + + /// TODO: Check if this cache is safe to use in case of different settings + + auto storage_columns_it = storage_id_to_columns_cache.find(table_node->getStorageID()); + if (storage_columns_it == storage_id_to_columns_cache.end()) + { + StorageColumns storage_columns; + storage_columns.column_names_and_types = table_node->getStorageSnapshot()->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); + + for (auto & name_and_type_pair : storage_columns.column_names_and_types) + { + Identifier column_name_identifier(name_and_type_pair.name); + storage_columns.column_identifier_first_parts.insert(column_name_identifier.at(0)); + } + + auto [inserted_it, _] = storage_id_to_columns_cache.emplace(storage_id, std::move(storage_columns)); + storage_columns_it = inserted_it; + } + + auto & storage_columns = storage_columns_it->second; + + /** If identifier first part binds to some column start. Then we can try to find whole identifier in table. + * 1. Try to bind identifier first part to column in table, if true get full identifier from table or throw exception. + * 2. Try to bind identifier first part to table name or table alias, if true remove first part and try to get full identifier from table or throw exception. + * 3. Try to bind identifier first parts to database name and table name, if true remove first two parts and try to get full identifier from table or throw exception. + */ + auto resolve_identifier_from_table_or_throw = [&](size_t drop_first_parts_size) + { + auto identifier_view = IdentifierView(identifier); + identifier_view.popFirst(drop_first_parts_size); + + auto result_column = storage_columns.column_names_and_types.tryGetByName(std::string(identifier_view.getFullName())); + if (!result_column) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Identifier {} cannot be resolved from table {}. In scope {}", + identifier.getFullName(), + storage_id.getFullTableName(), + scope.scope_node->formatASTForErrorMessage()); + + return result_column; + }; + + if (storage_columns.canBindIdentifier(IdentifierView(identifier))) + { + auto result_column = resolve_identifier_from_table_or_throw(0 /*drop_first_parts_size*/); + return std::make_shared(*result_column, from_node); + } + + if (identifier.getPartsSize() == 1) + return {}; + + if (path_start == table_name || (table_node->hasAlias() && path_start == table_node->getAlias())) + { + auto result_column = resolve_identifier_from_table_or_throw(1 /*drop_first_parts_size*/); + return std::make_shared(*result_column, from_node); + } + + if (identifier.getPartsSize() == 2) + return {}; + + if (path_start == database_name && identifier[1] == table_name) + { + auto result_column = resolve_identifier_from_table_or_throw(2 /*drop_first_parts_size*/); + return std::make_shared(*result_column, from_node); + } + + return {}; +} + +/** Resolve identifier in current scope. + * 1. Try resolve identifier from expression arguments. + * If prefer_column_name_to_alias = true. + * 2. Try to resolve identifier from tables. + * 3. Try to resolve identifier from aliases. + * Otherwise. + * 2. Try to resolve identifier from aliases. + * 3. Try to resolve identifier from tables. + */ +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierInCurrentScope(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & current_scope) +{ + auto resolved_identifier = tryResolveIdentifierFromExpressionArguments(identifier_lookup, current_scope); + if (resolved_identifier) + return resolved_identifier; + + bool prefer_column_name_to_alias = context->getSettingsRef().prefer_column_name_to_alias; + if (unlikely(prefer_column_name_to_alias)) + { + resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, current_scope); + if (resolved_identifier) + return resolved_identifier; + + return tryResolveIdentifierFromAliases(identifier_lookup, current_scope); + } + + resolved_identifier = tryResolveIdentifierFromAliases(identifier_lookup, current_scope); + if (resolved_identifier) + return resolved_identifier; + + return tryResolveIdentifierFromTables(identifier_lookup, current_scope); +} + +/** Try resolve identifier in current scope parent scopes. + * If initial scope is query. Then return nullptr. + * TODO: CTE, constants can be used from parent query with statement. + * TODO: If column is matched, throw exception that nested subqueries are not supported. + * + * If iniital scope is expression. Then try to resolve identifier in parent scopes until query scope is hit. + * For query scope resolve strategy is same as if initial scope if query. + */ +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierInParentScopes(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +{ + if (!scope.allow_to_check_parent_scopes) + return {}; + + bool initial_scope_is_query = scope.scope_node->getNodeType() == QueryTreeNodeType::QUERY; + bool initial_scope_is_expression = !initial_scope_is_query; + + if (initial_scope_is_expression) + { + IdentifierResolveScope * scope_to_check = scope.parent_scope; + while (scope_to_check != nullptr) + { + auto resolved_identifier = tryResolveIdentifier(identifier_lookup, *scope_to_check); + if (resolved_identifier) + return resolved_identifier; + + if (scope_to_check->scope_node->getNodeType() == QueryTreeNodeType::QUERY) + break; + + scope_to_check = scope_to_check->parent_scope; + } + } + + return {}; +} + +/** Resolve identifier in scope. + * Steps: + * 1. Register identifier lookup in scope identifier_lookup_to_resolve_status table. + * If entry is already registered and is not resolved, that means that we have cyclic aliases for identifier. + * Example: SELECT a AS b, b AS a; + * 2. Try resolve identifier in current scope. + * 3. If identifier is not resolved in current scope, try to resolve it in parent scopes. + * 4. If identifier was not resolved remove it from identifier_lookup_to_resolve_status table. + * It is okay for identifier to be not resolved, in case we want first try to lookup identifier in one context, + * then if there is no identifier in this context, try to lookup in another context. + * Example: Try to lookup identifier as function, if it is not found lookup as expression. + * Example: Try to lookup identifier as expression, if it is not found lookup as table. + */ +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifier(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +{ + auto it = scope.identifier_lookup_to_node.find(identifier_lookup); + if (it != scope.identifier_lookup_to_node.end()) + { + if (!it->second) + throw Exception(ErrorCodes::CYCLIC_ALIASES, + "Cyclic aliases for identifier {}. In scope {}", + identifier_lookup.identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + + if (scope.non_cached_identifier_lookups_during_expression_resolve.count(identifier_lookup) == 0) + return it->second; + } + + auto [insert_it, _] = scope.identifier_lookup_to_node.insert({identifier_lookup, QueryTreeNodePtr()}); + it = insert_it; + + QueryTreeNodePtr resolved_identifier = tryResolveIdentifierInCurrentScope(identifier_lookup, scope); + + if (!resolved_identifier) + resolved_identifier = tryResolveIdentifierInParentScopes(identifier_lookup, scope); + + it->second = resolved_identifier; + + if (!resolved_identifier || scope.non_cached_identifier_lookups_during_expression_resolve.count(identifier_lookup) > 0) + scope.identifier_lookup_to_node.erase(it); + + return resolved_identifier; +} + +/// Resolve query tree nodes functions implementation + +/** Resolve query tree matcher. Check MatcherNode.h for detailed matcher description. + * + * 1. Populate matcher_expression_nodes. + * + * If we resolve qualified matcher, first try to match qualified identifier to expression. If qualified identifier matched expression node then + * if expression is compound match it column names using matcher `isMatchingColumn` method, if expression is not compound, throw exception. + * If qualified identifier did not match expression in query tree, try to lookup qualified identifier in table context. + * + * If we resolve non qualified matcher, use current scope join tree node. + * + * 2. Apply column transforms to matched expression nodes. + */ +QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope) +{ + auto & matcher_node_typed = matcher_node->as(); + + std::vector matcher_expression_nodes; + + if (matcher_node_typed.isQualified()) + { + auto expression_query_tree_node = tryResolveIdentifier({matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::EXPRESSION}, scope); + if (expression_query_tree_node) + { + auto result_type = expression_query_tree_node->getResultType(); + + while (const auto * array_type = typeid_cast(result_type.get())) + result_type = array_type->getNestedType(); + + const auto * tuple_data_type = typeid_cast(result_type.get()); + if (!tuple_data_type) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Qualified matcher {} find non compound expression {} with type {}. Expected tuple or array of tuples. In scope {}", + matcher_node->formatASTForErrorMessage(), + expression_query_tree_node->formatASTForErrorMessage(), + expression_query_tree_node->getResultType()->getName(), + scope.scope_node->formatASTForErrorMessage()); + + const auto & element_names = tuple_data_type->getElementNames(); + + for (const auto & element_name : element_names) + { + auto tuple_element_function = std::make_shared("tupleElement"); + tuple_element_function->getArguments().getNodes().push_back(expression_query_tree_node); + tuple_element_function->getArguments().getNodes().push_back(std::make_shared(element_name)); + + QueryTreeNodePtr function_query_node = tuple_element_function; + resolveFunction(function_query_node, scope); + matcher_expression_nodes.push_back(std::move(function_query_node)); + } + } + else + { + auto table_query_tree_node = tryResolveIdentifier({matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::TABLE}, scope); + + if (!table_query_tree_node || table_query_tree_node->getNodeType() != QueryTreeNodeType::TABLE) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Qualified matcher {} does not find table. In scope {}", + matcher_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + const auto & table_node = table_query_tree_node->as(); + auto initial_matcher_columns = table_node.getStorageSnapshot()->getColumns(GetColumnsOptions(GetColumnsOptions::All)); + + for (auto & column : initial_matcher_columns) + { + const auto & column_name = column.name; + if (matcher_node_typed.isMatchingColumn(column_name)) + matcher_expression_nodes.push_back(std::make_shared(column, table_query_tree_node)); + } + } + } + else + { + /** There can be edge case if matcher is inside lambda expression. + * Try to find parent query expression using parent scopes. + */ + IQueryTreeNode * scope_node = scope.scope_node.get(); + auto * scope_query_node = scope_node->as(); + while (scope.scope_node && !scope_query_node) + { + if (scope.parent_scope) + scope_node = scope.parent_scope->scope_node.get(); + } + + /// If there are no parent scope that has tables or query scope does not have FROM section + if (!scope_query_node || !scope_query_node->getFrom()) + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Unqualified matcher {} cannot be resolved. There are no table sources. In scope {}", + matcher_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + + const auto & table_node = scope_query_node->getFrom()->as(); + + UInt8 get_column_options_kind = 0; + + if (matcher_node_typed.isAsteriskMatcher()) + { + get_column_options_kind = GetColumnsOptions::Ordinary; + + if (context->getSettingsRef().asterisk_include_alias_columns) + get_column_options_kind |= GetColumnsOptions::Kind::Aliases; + + if (context->getSettingsRef().asterisk_include_materialized_columns) + get_column_options_kind |= GetColumnsOptions::Kind::Materialized; + } + else + { + /// TODO: Check if COLUMNS select aliases column by default + get_column_options_kind = GetColumnsOptions::All; + } + + auto get_columns_options = GetColumnsOptions(static_cast(get_column_options_kind)); + auto initial_matcher_columns = table_node.getStorageSnapshot()->getColumns(get_columns_options); + + for (auto & column : initial_matcher_columns) + { + const auto & column_name = column.name; + if (matcher_node_typed.isMatchingColumn(column_name)) + matcher_expression_nodes.push_back(std::make_shared(column, scope_query_node->getFrom())); + } + } + + ListNodePtr list = std::make_shared(); + for (auto & node : matcher_expression_nodes) + { + for (const auto & transformer : matcher_node_typed.getColumnTransformers().getNodes()) + { + if (auto * apply_transformer = transformer->as()) + { + const auto & expression_node = apply_transformer->getExpressionNode(); + + if (apply_transformer->getApplyTransformerType() == ApplyColumnTransformerType::LAMBDA) + { + auto lambda_expression_to_resolve = expression_node->clone(); + IdentifierResolveScope lambda_scope(expression_node, &scope /*parent_scope*/); + resolveLambda(lambda_expression_to_resolve, {node}, lambda_scope); + auto & lambda_expression_to_resolve_typed = lambda_expression_to_resolve->as(); + node = lambda_expression_to_resolve_typed.getExpression(); + } + else if (apply_transformer->getApplyTransformerType() == ApplyColumnTransformerType::FUNCTION) + { + auto function_to_resolve_untyped = expression_node->clone(); + auto & function_to_resolve_typed = function_to_resolve_untyped->as(); + function_to_resolve_typed.getArguments().getNodes().push_back(node); + resolveFunction(function_to_resolve_untyped, scope); + node = function_to_resolve_untyped; + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Unsupported apply matcher expression type. Expected lambda or function apply transformer. Actual {}. In scope {}", + transformer->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + } + else if (auto * except_transformer = transformer->as()) + { + auto node_name = node->getName(); + if (except_transformer->isColumnMatching(node_name)) + { + node = {}; + break; + } + } + else if (auto * replace_transformer = transformer->as()) + { + auto node_name = node->getName(); + auto replace_expression = replace_transformer->findReplacementExpression(node_name); + if (!replace_expression) + continue; + + node = replace_expression; + resolveExpressionNode(node, scope, false /*allow_lambda_expression*/); + } + } + + if (node) + list->getNodes().push_back(node); + } + + return list; +} + + +/** Resolve lambda function. + * This function modified lambda_node during resolve. It is caller responsibility to clone lambda before resolve + * if it is needed for later use. + * + * lambda_node - node that must have LambdaNode type. + * arguments - lambda arguments. + * scope - lambda scope. It is client responsibility to create it. + * + * Resolve steps: + * 1. Valide arguments. + * 2. Register lambda in lambdas in resolve process. This is necessary to prevent recursive lambda resolving. + * 3. Initialize scope with lambda aliases. + * 4. Validate lambda argument names, and scope expressions. + * 5. Resolve lambda body expression. + * 6. Deregister lambda from lambdas in resolve process. + */ +void QueryAnalyzer::resolveLambda(QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope) +{ + auto & lambda = lambda_node->as(); + auto & lambda_arguments_nodes = lambda.getArguments().getNodes(); + size_t lambda_argument_nodes_size = lambda_arguments_nodes.size(); + + /** Register lambda as being resolved, to prevent recursive lambdas resolution. + * Example: WITH (x -> x + lambda_2(x)) AS lambda_1, (x -> x + lambda_1(x)) AS lambda_2 SELECT 1; + */ + auto it = lambdas_in_resolve_process.find(lambda_node.get()); + if (it != lambdas_in_resolve_process.end()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Recursive lambda {}. In scope {}", + lambda.formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + size_t arguments_size = lambda_arguments.size(); + if (lambda_argument_nodes_size != arguments_size) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Lambda {} expect {} arguments. Actual {}. In scope {}", + lambda.formatASTForErrorMessage(), + arguments_size, + lambda_argument_nodes_size, + scope.scope_node->formatASTForErrorMessage()); + + /// Initialize aliases in lambda scope + ScopeAliasVisitorMatcher::Data data{scope}; + ScopeAliasVisitorMatcher::Visitor visitor(data); + visitor.visit(lambda_node); + + /** Replace lambda arguments with new arguments. + * Additionally validate that there are no aliases with same name as lambda arguments. + * Arguments are registered in current scope expression_argument_name_to_node map. + */ + auto lambda_new_arguments = std::make_shared(); + lambda_new_arguments->getNodes().reserve(lambda_argument_nodes_size); + + for (size_t i = 0; i < lambda_argument_nodes_size; ++i) + { + auto & lambda_argument_node = lambda_arguments_nodes[i]; + auto & lambda_argument_node_typed = lambda_argument_node->as(); + const auto & lambda_argument_name = lambda_argument_node_typed.getIdentifier().getFullName(); + + bool has_expression_node = data.scope.alias_name_to_expression_node.count(lambda_argument_name) > 0; + bool has_alias_node = data.scope.alias_name_to_lambda_node.count(lambda_argument_name) > 0; + + if (has_expression_node || has_alias_node) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Alias name {} inside lambda cannot have same name as lambda argument. In scope {}", + lambda_argument_name, + lambda_argument_node_typed.formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + + scope.expression_argument_name_to_node.emplace(lambda_argument_name, lambda_arguments[i]); + lambda_new_arguments->getNodes().push_back(lambda_arguments[i]); + } + + lambda.getArgumentsNode() = std::move(lambda_new_arguments); + + /** Lambda body expression is resolved as standard query expression node. + * After that lambda is resolved, because its expression node is resolved. + */ + resolveExpressionNode(lambda.getExpression(), scope, false /*allow_lambda_expression*/); + + /** TODO: Lambda body can be resolved in expression list. And for standalone lambdas it will work. + * TODO: It can potentially be resolved into table or another lambda. + * Example: WITH (x -> untuple(x)) AS lambda SELECT untuple(compound_expression). + */ + // if (lambda.getExpression()->getNodeType() == QueryTreeNodeType::LIST) + // throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + // "Lambda {} expression body cannot contain list of expressions. In scope {}", + // lambda_node->formatASTForErrorMessage(), + // scope.scope_node->formatASTForErrorMessage()); + + lambdas_in_resolve_process.erase(lambda_node.get()); +} + +/** Resolve function node in scope. + * During function node resolve, function node can be replaced with another expression (if it match lambda or sql user defined function), + * with constant (if it allow constant folding), or with expression list. It is caller responsibility to handle such cases appropriately. + * + * Steps: + * 1. Resolve function parameters. Validate that each function parameter must be constant node. + * 2. Resolve function arguments list, lambda expressions are allowed as function arguments. + * 3. Initialize argument_columns, argument_types, function_lambda_arguments_indexes arrays from function arguments. + * 4. Try to resolve function name as identifier as function. + * 5. If function name identifier was not resolved as function, try to lookup lambda from sql user defined functions factory. + * 6. If function was resolve as lambda from step 4, or 5, then resolve lambda using function arguments and replace function node with lambda result. + * After than function node is resolved. + * 7. If function was not resolved during step 6 as lambda, then try to resolve function as executable user defined function or aggregate function or + * non aggregate function. + * + * Special case is `untuple` function that takes single compound argument expression. If argument is not compound expression throw exception. + * Wrap compound expression subcolumns into `tupleElement` and replace function node with them. After that `untuple` function node is resolved. + * + * If function is resolved as executable user defined function or aggregate function, function node is resolved + * no additional special handling is required. + * + * 8. If function was resolved as non aggregate function. Then if on step 3 there were lambdas, their result types need to be initialized and + * they must be resolved. + * 9. If function is suitable for constant folding, try to replace function node with constant result. + * + * TODO: Special `in` function. + * TODO: Special `grouping` function. + * TODO: Window functions. + */ +void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveScope & scope) +{ + FunctionNode & function_node = node->as(); + if (function_node.isResolved()) + return; + + /// Resolve function parameters + + resolveExpressionNodeList(function_node.getParametersNode(), scope, false /*allow_lambda_expression*/); + + /// Convert function parameters into constant parameters array + + Array parameters; + + auto & parameters_nodes = function_node.getParameters().getNodes(); + parameters.reserve(parameters_nodes.size()); + + for (auto & parameter : parameters_nodes) + { + auto * constant_parameter = parameter->as(); + if (constant_parameter) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Parameter for function {} expected to be constant expression. Actual {}. In scope {}", + function_node.getFunctionName(), + parameter->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + parameters.push_back(constant_parameter); + } + + /// Resolve function arguments + + resolveExpressionNodeList(function_node.getArgumentsNode(), scope, true /*allow_lambda_expression*/); + + /// Initialize function argument columns + + ColumnsWithTypeAndName argument_columns; + DataTypes argument_types; + bool all_arguments_constants = true; + std::vector function_lambda_arguments_indexes; + + auto & function_arguments = function_node.getArguments().getNodes(); + size_t function_arguments_size = function_arguments.size(); + + for (size_t function_argument_index = 0; function_argument_index < function_arguments_size; ++function_argument_index) + { + auto & function_argument = function_arguments[function_argument_index]; + + ColumnWithTypeAndName argument_column; + argument_column.name = function_argument->getName(); + + /** If function argument is lambda, save lambda argument index and initialize argument type as DataTypeFunction + * where function argument types are initialized with empty array of lambda arguments size. + */ + if (auto * lambda_query_tree_node = function_argument->as()) + { + size_t lambda_arguments_size = lambda_query_tree_node->getArguments().getNodes().size(); + argument_column.type = std::make_shared(DataTypes(lambda_arguments_size, nullptr), nullptr); + function_lambda_arguments_indexes.push_back(function_argument_index); + } + else + { + argument_column.type = function_argument->getResultType(); + } + + if (!argument_column.type) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Function {} argument is not resolved. In scope {}", + function_node.getFunctionName(), + scope.scope_node->formatASTForErrorMessage()); + + if (auto * constant_query_tree_node = function_argument->as()) + argument_column.column = argument_column.type->createColumnConst(1, constant_query_tree_node->getConstantValue()); + else + all_arguments_constants = false; + + argument_types.push_back(argument_column.type); + argument_columns.emplace_back(std::move(argument_column)); + } + + /** Lookup function node name as lambda identifier. + * If no lambda node exists with function node name identifier, try to resolve it as lambda from sql user defined functions. + */ + auto lambda_expression_untyped = tryResolveIdentifier({Identifier{function_node.getFunctionName()}, IdentifierLookupContext::FUNCTION}, scope); + // if (!lambda_expression_untyped) + // lambda_expression_untyped = tryGetLambdaFromSQLUserDefinedFunction(function_node.getFunctionName()); + + /** If function is resolved as lambda. + * Clone lambda before resolve. + * Initialize lambda arguments as function arguments + * Resolve lambda and then replace function node with resolved lambda expression body. + * Example: WITH (x -> x + 1) AS lambda SELECT lambda(value) FROM test_table; + * Result: SELECT value + 1 FROM test_table; + */ + if (lambda_expression_untyped) + { + auto * lambda_expression = lambda_expression_untyped->as(); + if (!lambda_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Function identifier {} must be resolved as lambda. Actual {}. In scope {}", + function_node.getFunctionName(), + lambda_expression_untyped->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + auto lambda_expression_clone = lambda_expression_untyped->clone(); + + IdentifierResolveScope lambda_scope(lambda_expression_clone, &scope /*parent_scope*/); + resolveLambda(lambda_expression_clone, function_arguments, lambda_scope); + + auto & resolved_lambda = lambda_expression_clone->as(); + node = resolved_lambda.getExpression(); + return; + } + + const auto & function_name = function_node.getFunctionName(); + + /// Special handling of `untuple` function + + if (function_name == "untuple") + { + if (function_arguments.size() != 1) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Function 'untuple' must have 1 argument. In scope {}", + scope.scope_node->formatASTForErrorMessage()); + + const auto & tuple_argument = function_arguments[0]; + auto result_type = tuple_argument->getResultType(); + const auto * tuple_data_type = typeid_cast(result_type.get()); + if (!tuple_data_type) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Function untuple argument must be have compound type. Actual type {}. In scope {}", + result_type->getName(), + scope.scope_node->formatASTForErrorMessage()); + + const auto & element_names = tuple_data_type->getElementNames(); + + auto result_list = std::make_shared(); + result_list->getNodes().reserve(element_names.size()); + + for (const auto & element_name : element_names) + { + auto tuple_element_function = std::make_shared("tupleElement"); + tuple_element_function->getArguments().getNodes().push_back(tuple_argument); + tuple_element_function->getArguments().getNodes().push_back(std::make_shared(element_name)); + + QueryTreeNodePtr function_query_node = tuple_element_function; + resolveFunction(function_query_node, scope); + result_list->getNodes().push_back(std::move(function_query_node)); + } + + node = result_list; + return; + } + + /** Try to resolve function as + * 1. Executable user defined function. + * 2. Aggregate function. + * 3. Non aggregate function. + * TODO: Provide better hints. + */ + FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, context, parameters); + + if (!function) + function = FunctionFactory::instance().tryGet(function_name, context); + + if (!function) + { + if (!AggregateFunctionFactory::instance().isAggregateFunctionName(function_name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Function with name {} does not exists. In scope {}", + function_name, + scope.scope_node->formatASTForErrorMessage()); + + AggregateFunctionProperties properties; + auto aggregate_function = AggregateFunctionFactory::instance().get(function_name, argument_types, parameters, properties); + function_node.resolveAsAggregateFunction(aggregate_function, aggregate_function->getReturnType()); + return; + } + + /** For lambda arguments we need to initialize lambda argument types DataTypeFunction using `getLambdaArgumentTypes` function. + * Then each lambda arguments are initalized with columns, where column source is lambda. + * This information is important for later steps of query processing. + * Example: SELECT arrayMap(x -> x + 1, [1, 2, 3]). + * lambda node x -> x + 1 identifier x is resolved as column where source is lambda node. + */ + bool has_lambda_arguments = !function_lambda_arguments_indexes.empty(); + if (has_lambda_arguments) + { + function->getLambdaArgumentTypes(argument_types); + + for (auto & function_lambda_argument_index : function_lambda_arguments_indexes) + { + auto lambda_argument_clone = function_arguments[function_lambda_argument_index]->clone(); + auto & lambda_argument_clone_typed = lambda_argument_clone->as(); + const auto & lambda_argument_clone_argument_names = lambda_argument_clone_typed.getArgumentNames(); + + size_t lambda_arguments_size = lambda_argument_clone_typed.getArguments().getNodes().size(); + + const auto * function_data_type = typeid_cast(argument_types[function_lambda_argument_index].get()); + if (!function_data_type) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Function {} expected function data type for lambda argument with index {}. Actual {}. In scope {}", + function_name, + function_lambda_argument_index, + argument_types[function_lambda_argument_index]->getName(), + scope.scope_node->formatASTForErrorMessage()); + + const auto & function_data_type_argument_types = function_data_type->getArgumentTypes(); + size_t function_data_type_arguments_size = function_data_type_argument_types.size(); + if (function_data_type_arguments_size != lambda_arguments_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Function {} function data type for lambda argument wiht index {} arguments size mismatch. Actual {}. Expected {}. In scope {}", + function_name, + function_data_type_arguments_size, + lambda_arguments_size, + argument_types[function_lambda_argument_index]->getName(), + scope.scope_node->formatASTForErrorMessage()); + + QueryTreeNodes lambda_arguments; + lambda_arguments.reserve(lambda_arguments_size); + + for (size_t i = 0; i < lambda_arguments_size; ++i) + { + const auto & argument_type = function_data_type_argument_types[i]; + auto column_name_and_type = NameAndTypePair{lambda_argument_clone_argument_names[i], argument_type}; + lambda_arguments.push_back(std::make_shared(std::move(column_name_and_type), lambda_argument_clone)); + } + + IdentifierResolveScope lambda_scope(lambda_argument_clone, &scope /*parent_scope*/); + resolveLambda(lambda_argument_clone, lambda_arguments, lambda_scope); + + argument_types[function_lambda_argument_index] = std::make_shared(function_data_type_argument_types, lambda_argument_clone->getResultType()); + argument_columns[function_lambda_argument_index].type = argument_types[function_lambda_argument_index]; + function_arguments[function_lambda_argument_index] = std::move(lambda_argument_clone); + } + } + + DataTypePtr result_type; + + try + { + auto function_base = function->build(argument_columns); + result_type = function_base->getResultType(); + + /** If function is suitable for constant folding try to convert it to constant. + * Example: SELECT plus(1, 1); + * Result: SELECT 2; + */ + if (function_base->isSuitableForConstantFolding()) + { + auto executable_function = function_base->prepare(argument_columns); + + ColumnPtr column; + + if (all_arguments_constants) + { + size_t num_rows = function_arguments.empty() ? 0 : argument_columns.front().column->size(); + column = executable_function->execute(argument_columns, result_type, num_rows, true); + } + else + { + column = function_base->getConstantResultForNonConstArguments(argument_columns, result_type); + } + + if (column && isColumnConst(*column)) + { + /// Replace function node with result constant node + Field constant_value; + column->get(0, constant_value); + node = std::make_shared(std::move(constant_value), result_type); + return; + } + } + } + catch (Exception & e) + { + e.addMessage("In scope {}", scope.scope_node->formatASTForErrorMessage()); + throw; + } + + function_node.resolveAsFunction(std::move(function), std::move(result_type)); +} + +/** Resolve expression node. + * Argument node can be replaced with different node, or even with list node in case of mather resolution. + * Example: SELECT * FROM test_table; + * * - is matcher node, and it can be resolved into ListNode. + * + * Steps: + * 1. If node has alias, replace node with its value in scope alias map. Register alias in expression_aliases_in_resolve_process, to prevent resolving identifier + * which can bind to expression alias name. Check tryResolveIdentifierFromAliases documentation for additional explanation. + * Example: + * SELECT id AS id FROM test_table; + * SELECT value.value1 AS value FROM test_table; + * + * 2. Call specific resolve method depending on node type. + * TODO: Need a way to prevent scalar subquery evaluation, for special functions like `in`. + * 3. Special case identifier node. Try first resolve it as expression identifier. Then if allow_lambda_expression = true + * try to resolve it as function. TODO: Add tables. + * 4. If node has alias, update its value in scope alias map. Deregister alias from expression_aliases_in_resolve_process. + */ +void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression) +{ + String node_alias; + + if (node->hasAlias()) + { + /** Node could be potentially resolved by resolving other nodes. + * SELECT b, a as b FROM test_table; + * + * To resolve b we need to resolve a. + */ + node_alias = node->getAlias(); + auto it = scope.alias_name_to_expression_node.find(node_alias); + if (it != scope.alias_name_to_expression_node.end()) + node = it->second; + + if (allow_lambda_expression) + { + it = scope.alias_name_to_lambda_node.find(node_alias); + if (it != scope.alias_name_to_lambda_node.end()) + node = it->second; + } + } + + scope.expressions_in_resolve_process_stack.pushNode(node); + + if (auto * identifier_node = node->as()) + { + auto unresolved_identifier = identifier_node->getIdentifier(); + node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::EXPRESSION}, scope); + + if (node && !node_alias.empty()) + scope.alias_name_to_lambda_node.erase(node_alias); + + if (!node && allow_lambda_expression) + { + node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::FUNCTION}, scope); + + if (node && !node_alias.empty()) + scope.alias_name_to_expression_node.erase(node_alias); + } + + if (!node) + { + std::string lambda_message_clarification; + if (allow_lambda_expression) + lambda_message_clarification = std::string(" or ") + toStringLowercase(IdentifierLookupContext::FUNCTION); + + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Unknown {}{} identifier {} in scope {}", + toStringLowercase(IdentifierLookupContext::EXPRESSION), + lambda_message_clarification, + unresolved_identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + } + } + else if (auto * function_node = node->as()) + { + resolveFunction(node, scope); + } + else if (auto * constant_function_node = node->as()) + { + /// Already resolved + } + else if (auto * column_function_node = node->as()) + { + /// Already resolved + } + else if (auto * lambda_node = node->as()) + { + if (!allow_lambda_expression) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Lambda is not allowed {} in expression context. In scope {}", + lambda_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + /// Must be resolved by caller + } + else if (auto * asterisk_node = node->as()) + { + node = resolveMatcher(node, scope); + } + else if (auto * query_node = node->as()) + { + IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); + resolveQuery(node, subquery_scope); + evaluateScalarSubquery(node); + } + else if (auto * list_node = node->as()) + { + /** Edge case if list expression has alias. + * Matchers cannot have aliases, but `untuple` function can. + * Example: SELECT a, untuple(CAST(('hello', 1) AS Tuple(name String, count UInt32))) AS a; + * During resolveFunction `untuple` function is replaced by list of 2 constants 'hello', 1. + */ + resolveExpressionNodeList(node, scope, allow_lambda_expression); + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Node with type {} is not supported {}. In scope {}", + node->getNodeTypeName(), + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + + /// Update aliases after expression node was resolved + if (!node_alias.empty()) + { + auto it = scope.alias_name_to_expression_node.find(node_alias); + if (it != scope.alias_name_to_expression_node.end()) + it->second = node; + + if (allow_lambda_expression) + { + it = scope.alias_name_to_lambda_node.find(node_alias); + if (it != scope.alias_name_to_lambda_node.end()) + it->second = node; + } + } + + scope.expressions_in_resolve_process_stack.popNode(); + bool expression_was_root = scope.expressions_in_resolve_process_stack.empty(); + if (expression_was_root) + scope.non_cached_identifier_lookups_during_expression_resolve.clear(); +} + +/** Resolve expression node list. + * If expression is resolved in list, it is flattened into initial node list. + * + * Such examples must work: + * Example: CREATE TABLE test_table (id UInt64, value UInt64) ENGINE=TinyLog; SELECT plus(*) FROM test_table; + * Example: SELECT *** FROM system.one; + */ +void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression) +{ + auto & initial_node_list = node_list->as(); + size_t initial_node_list_size = initial_node_list.getNodes().size(); + + auto result_node_list = std::make_shared(); + result_node_list->getNodes().reserve(initial_node_list_size); + + for (auto & node : initial_node_list.getNodes()) + { + resolveExpressionNode(node, scope, allow_lambda_expression); + + if (auto * expression_list = node->as()) + { + for (auto & expression_list_node : expression_list->getNodes()) + result_node_list->getNodes().push_back(std::move(expression_list_node)); + } + else + { + result_node_list->getNodes().push_back(std::move(node)); + } + } + + node_list = std::move(result_node_list); +} + +/** Resolve query. + * This function modifies query node during resolve. It is caller responsibility to clone query node before resolve + * if it is needed for later use. + * + * lambda_node - query_tree_node that must have QueryNode type. + * scope - query scope. It is caller responsibility to create it. + * + * Resolve steps: + * 1. Initialize query scope with aliases. + * 2. Resolve expressions in query parts. + * 3. Remove WITH section from query. + */ +void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_tree_node, IdentifierResolveScope & scope) +{ + auto & query_tree = query_tree_node->as(); + + /// Initialize aliases in query node scope + + ScopeAliasVisitorMatcher::Data data{scope}; + ScopeAliasVisitorMatcher::Visitor visitor(data); + + if (query_tree.getWithNode()) + visitor.visit(query_tree.getWithNode()); + + if (query_tree.getProjectionNode()) + visitor.visit(query_tree.getProjectionNode()); + + if (query_tree.getPrewhere()) + visitor.visit(query_tree.getPrewhere()); + + if (query_tree.getWhere()) + visitor.visit(query_tree.getWhere()); + + auto from_node_alias = query_tree.getFrom()->getAlias(); + if (!from_node_alias.empty()) + { + auto [_, inserted] = scope.alias_name_to_table_expression_node.insert(std::make_pair(from_node_alias, query_tree.getFrom())); + if (!inserted) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Multiple alises does not point to same entity {}", from_node_alias); + } + + /** Resolve query node sections. + * + * WITH section is not resolved at all and will be removed after query analysis pass. + * WITH section only can provide aliases to expressions and CTE for other sections to use. + * + * Example: WITH 1 AS constant, (x -> x + 1) AS lambda, a AS (SELECT * FROM test_table); + */ + resolveExpressionNodeList(query_tree.getProjectionNode(), scope, false /*allow_lambda_expression*/); + + if (query_tree.getPrewhere()) + resolveExpressionNode(query_tree.getPrewhere(), scope, false /*allow_lambda_expression*/); + + if (query_tree.getWhere()) + resolveExpressionNode(query_tree.getWhere(), scope, false /*allow_lambda_expression*/); + + /// Remove WITH section + query_tree.getWithNode() = std::make_shared(); +} + +void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + if (query_tree_node->getNodeType() != QueryTreeNodeType::QUERY) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "QueryAnalysis pass requires query tree node"); + + QueryAnalyzer analyzer(std::move(context)); + analyzer.resolve(query_tree_node); +} + +} diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h new file mode 100644 index 00000000000..8dc8e255e53 --- /dev/null +++ b/src/Analyzer/QueryAnalysisPass.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/** This pass make initial query analysis. + * + * 1. All identifiers are resolved. Next passes can expect that there will be no IdentifierNode in query tree. + * 2. All matchers are resolved. Next passes can expect that there will be no MatcherNode in query tree. + * 3. All functions are resolved. Next passes can expect that for each FunctionNode its result type will be set, and it will be resolved + * as aggregate or non aggregate function. + * 4. All lambda expressions that are function arguments are resolved. Next passes can expect that LambaNode expression is resolved, and lambda has concrete arguments. + * 5. All standalone lambda expressions are resolved. Next passes can expect that there will be no standalone LambaNode expressions in query. + * 6. Constants are folded. Example: SELECT plus(1, 1). After step will be: SELECT 2. + * Motivation for this, there are places in query tree that must contain constant: + * Function parameters Example: SELECT quantile(0.5)(x). + * Functions in which result type depends on constant expression. Example: cast(x, 'type_name'). + * Expressions that are part of LIMIT. Example: SELECT * FROM test_table LIMIT expr. + * + * 7. All scalar subqueries are evaluated. + * TODO: Scalar subqueries must be evaluated only if they are part of query tree where we must have constant. This is currently not done + * because execution layer does not support scalar subqueries execution. + * + * 8. Special function `untuple` is handled properly. + * 9. Sets are build for `in` function. + */ +class QueryAnalysisPass final : public IQueryTreePass +{ +public: + String getName() override + { + return "QueryAnalysis"; + } + + String getDescription() override + { + return "Resolve type for each query expression. Replace identifiers, matchers with query expressions. Perform constant folding. Evaluate scalar subqueries."; + } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp new file mode 100644 index 00000000000..e55bd4fc178 --- /dev/null +++ b/src/Analyzer/QueryNode.cpp @@ -0,0 +1,100 @@ +#include + +#include +#include + +#include +#include +#include + +namespace DB +{ + +QueryNode::QueryNode() +{ + children.resize(children_size); + children[with_child_index] = std::make_shared(); + children[projection_child_index] = std::make_shared(); +} + +void QueryNode::dumpTree(WriteBuffer & buffer, size_t indent) const +{ + buffer << std::string(indent, ' ') << "QUERY "; + writePointerHex(this, buffer); + buffer << '\n'; + + if (!getWith().getNodes().empty()) + { + buffer << std::string(indent, ' ') << "WITH\n"; + getWith().dumpTree(buffer, indent + 2); + buffer << '\n'; + } + + buffer << std::string(indent, ' ') << "PROJECTION\n"; + getProjection().dumpTree(buffer, indent + 2); + buffer << '\n'; + + if (getFrom()) + { + buffer << std::string(indent, ' ') << "JOIN TREE\n"; + getFrom()->dumpTree(buffer, indent + 2); + buffer << '\n'; + } + + if (getPrewhere()) + { + buffer << std::string(indent, ' ') << "PREWHERE\n"; + getPrewhere()->dumpTree(buffer, indent + 2); + buffer << '\n'; + } + + if (getWhere()) + { + buffer << std::string(indent, ' ') << "WHERE\n"; + getWhere()->dumpTree(buffer, indent + 2); + buffer << '\n'; + } +} + +void QueryNode::updateTreeHashImpl(HashState &) const +{ + /// TODO: No state +} + +ASTPtr QueryNode::toASTImpl() const +{ + auto select_query = std::make_shared(); + + if (!getWith().getNodes().empty()) + select_query->setExpression(ASTSelectQuery::Expression::WITH, getWithNode()->toAST()); + + select_query->setExpression(ASTSelectQuery::Expression::SELECT, children[projection_child_index]->toAST()); + + auto table_expression_ast = std::make_shared(); + table_expression_ast->children.push_back(children[from_child_index]->toAST()); + table_expression_ast->database_and_table_name = table_expression_ast->children.back(); + + auto tables_in_select_query_element_ast = std::make_shared(); + tables_in_select_query_element_ast->children.push_back(std::move(table_expression_ast)); + tables_in_select_query_element_ast->table_expression = tables_in_select_query_element_ast->children.back(); + + auto tables_in_select_query_ast = std::make_shared(); + tables_in_select_query_ast->children.push_back(std::move(tables_in_select_query_element_ast)); + + select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select_query_ast)); + + if (getPrewhere()) + select_query->setExpression(ASTSelectQuery::Expression::PREWHERE, getPrewhere()->toAST()); + + if (getWhere()) + select_query->setExpression(ASTSelectQuery::Expression::WHERE, getWhere()->toAST()); + + return select_query; +} + +QueryTreeNodePtr QueryNode::cloneImpl() const +{ + return std::make_shared(); +} + +} diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h new file mode 100644 index 00000000000..1ed59b9c5c8 --- /dev/null +++ b/src/Analyzer/QueryNode.h @@ -0,0 +1,118 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/** Query node represents query in query tree. + * TODO: CTE. + */ +class QueryNode; +using QueryNodePtr = std::shared_ptr; + +class QueryNode final : public IQueryTreeNode +{ +public: + QueryNode(); + + const ListNode & getWith() const + { + return children[with_child_index]->as(); + } + + ListNode & getWith() + { + return children[with_child_index]->as(); + } + + const QueryTreeNodePtr & getWithNode() const + { + return children[with_child_index]; + } + + QueryTreeNodePtr & getWithNode() + { + return children[with_child_index]; + } + + const ListNode & getProjection() const + { + return children[projection_child_index]->as(); + } + + ListNode & getProjection() + { + return children[projection_child_index]->as(); + } + + const QueryTreeNodePtr & getProjectionNode() const + { + return children[projection_child_index]; + } + + QueryTreeNodePtr & getProjectionNode() + { + return children[projection_child_index]; + } + + const QueryTreeNodePtr & getFrom() const + { + return children[from_child_index]; + } + + QueryTreeNodePtr & getFrom() + { + return children[from_child_index]; + } + + const QueryTreeNodePtr & getPrewhere() const + { + return children[prewhere_child_index]; + } + + QueryTreeNodePtr & getPrewhere() + { + return children[prewhere_child_index]; + } + + const QueryTreeNodePtr & getWhere() const + { + return children[where_child_index]; + } + + QueryTreeNodePtr & getWhere() + { + return children[where_child_index]; + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::QUERY; + } + + void dumpTree(WriteBuffer & buffer, size_t indent) const override; + +protected: + void updateTreeHashImpl(HashState &) const override; + + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + static constexpr size_t with_child_index = 0; + static constexpr size_t projection_child_index = 1; + static constexpr size_t from_child_index = 2; + static constexpr size_t prewhere_child_index = 3; + static constexpr size_t where_child_index = 4; + static constexpr size_t group_by_child_index = 5; + static constexpr size_t having_child_index = 6; + static constexpr size_t order_by_child_index = 7; + static constexpr size_t limit_child_index = 8; + static constexpr size_t children_size = where_child_index + 1; +}; + +} diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp new file mode 100644 index 00000000000..3c041635dad --- /dev/null +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -0,0 +1,550 @@ +#include + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; + extern const int UNKNOWN_DATABASE; + extern const int UNKNOWN_TABLE; + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; +} + +class QueryTreeBuilder : public WithContext +{ +public: + QueryTreeBuilder(ASTPtr query_, ContextPtr context_); + + QueryTreeNodePtr getQueryTreeNode() + { + return query_tree_node; + } + +private: + QueryTreeNodePtr getSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_scalar_query) const; + + QueryTreeNodePtr getSelectExpression(const ASTPtr & select_query, bool is_scalar_query) const; + + QueryTreeNodePtr getExpressionList(const ASTPtr & expression_list) const; + + QueryTreeNodePtr getExpression(const ASTPtr & expression) const; + + QueryTreeNodePtr getFromNode(const ASTPtr & tables_in_select_query) const; + + ColumnTransformersNodes getColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const; + + StoragePtr resolveStorage(const Identifier & storage_identifier) const; + + ASTPtr query; + QueryTreeNodePtr query_tree_node; + +}; + +QueryTreeBuilder::QueryTreeBuilder(ASTPtr query_, ContextPtr context_) + : WithContext(context_) + , query(query_->clone()) +{ + if (query->as()) + query_tree_node = getSelectWithUnionExpression(query, false /*is_scalar_query*/); + else if (query->as()) + query_tree_node = getSelectExpression(query, false /*is_scalar_query*/); + else if (query->as()) + query_tree_node = getExpressionList(query); + else + query_tree_node = getExpression(query); +} + +QueryTreeNodePtr QueryTreeBuilder::getSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_scalar_query) const +{ + auto & select_with_union_query_typed = select_with_union_query->as(); + auto & select_lists = select_with_union_query_typed.list_of_selects->as(); + + if (select_lists.children.size() == 1) + { + return getSelectExpression(select_with_union_query->children[0]->children[0], is_scalar_query); + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "UNION is not supported"); + // auto union_expression = UnionExpression::create(is_scalar_query); + + // union_expression->getModes() = select_with_union_query_typed.list_of_modes; + // union_expression->getModesSet() = select_with_union_query_typed.set_of_modes; + + // auto & select_expressions = union_expression->getSelectExpressions(); + // select_expressions.reserve(select_lists.children.size()); + + // for (const auto & select : select_lists.children) + // { + // auto expression = getSelectExpression(select, false); + // select_expressions.emplace_back(std::move(expression)); + // } + + // return union_expression; + } +} + +QueryTreeNodePtr QueryTreeBuilder::getSelectExpression(const ASTPtr & select_query, bool) const +{ + const auto & select_query_typed = select_query->as(); + auto current_query_tree = std::make_shared(); + current_query_tree->getFrom() = getFromNode(select_query_typed.tables()); + current_query_tree->setOriginalAST(select_query); + + auto select_with_list = select_query_typed.with(); + if (select_with_list) + { + auto & select_with_list_typed = select_with_list->as(); + for (auto & expression_part : select_with_list_typed.children) + { + auto expression_node = getExpression(expression_part); + current_query_tree->getWith().getNodes().push_back(expression_node); + } + } + + auto select_expression_list = select_query_typed.select(); + if (select_expression_list) + { + auto & select_expression_list_typed = select_expression_list->as(); + + for (auto & expression_part : select_expression_list_typed.children) + { + auto expression_node = getExpression(expression_part); + current_query_tree->getProjection().getNodes().push_back(expression_node); + } + } + + auto prewhere_expression = select_query_typed.prewhere(); + if (prewhere_expression) + current_query_tree->getPrewhere() = getExpression(prewhere_expression); + + auto where_expression = select_query_typed.where(); + if (where_expression) + current_query_tree->getWhere() = getExpression(where_expression); + + return current_query_tree; +} + +QueryTreeNodePtr QueryTreeBuilder::getExpressionList(const ASTPtr & expression_list) const +{ + auto list_node = std::make_shared(); + + auto & expression_list_typed = expression_list->as(); + list_node->getNodes().reserve(expression_list_typed.children.size()); + + for (auto & expression : expression_list_typed.children) + { + auto expression_node = getExpression(expression); + list_node->getNodes().push_back(std::move(expression_node)); + } + + return list_node; +} + +QueryTreeNodePtr QueryTreeBuilder::getExpression(const ASTPtr & expression) const +{ + QueryTreeNodePtr result; + + if (const auto * ast_identifier = expression->as()) + { + /// TODO: Identifier as query parameter + auto identifier = Identifier(ast_identifier->name_parts); + result = std::make_shared(std::move(identifier)); + } + else if (const auto * asterisk = expression->as()) + { + auto column_transformers = getColumnTransformers(expression, 0 /*start_child_index*/); + result = std::make_shared(column_transformers); + } + else if (const auto * qualified_asterisk = expression->as()) + { + /// TODO: Identifier with UUID + /// TODO: Currently during query analysis stage we support qualified matchers with any identifier length + /// but ASTTableIdentifier can contain only 2 parts. + + auto & qualified_identifier = qualified_asterisk->children.at(0)->as(); + auto column_transformers = getColumnTransformers(expression, 1 /*start_child_index*/); + result = std::make_shared(Identifier(qualified_identifier.name_parts), column_transformers); + } + else if (const auto * ast_literal = expression->as()) + { + result = std::make_shared(ast_literal->value); + } + else if (const auto * function = expression->as()) + { + if (function->is_lambda_function) + { + const auto & lambda_arguments_and_expression = function->arguments->as().children; + auto & lambda_arguments_tuple = lambda_arguments_and_expression.at(0)->as(); + + auto lambda_arguments_nodes = std::make_shared(); + Names lambda_arguments; + NameSet lambda_arguments_set; + + if (lambda_arguments_tuple.arguments) + { + const auto & lambda_arguments_list = lambda_arguments_tuple.arguments->as()->children; + for (const auto & lambda_argument : lambda_arguments_list) + { + const auto * lambda_argument_identifier = lambda_argument->as(); + + if (!lambda_argument_identifier) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Lambda {} argument is not identifier", + function->formatForErrorMessage()); + + if (lambda_argument_identifier->name_parts.size() > 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Lambda {} argument identifier must contain only argument name. Actual {}", + function->formatForErrorMessage(), + lambda_argument_identifier->full_name); + + const auto & argument_name = lambda_argument_identifier->name_parts[0]; + auto [_, inserted] = lambda_arguments_set.insert(argument_name); + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Lambda {} multiple arguments with same name {}", + function->formatForErrorMessage(), + argument_name); + + lambda_arguments.push_back(argument_name); + } + } + + const auto & lambda_expression = lambda_arguments_and_expression.at(1); + auto lambda_expression_node = getExpression(lambda_expression); + + result = std::make_shared(std::move(lambda_arguments), std::move(lambda_expression_node)); + } + else + { + auto function_node = std::make_shared(function->name); + + if (function->parameters) + { + const auto & function_parameters_list = function->parameters->as()->children; + for (const auto & argument : function_parameters_list) + function_node->getParameters().getNodes().push_back(getExpression(argument)); + } + + if (function->arguments) + { + const auto & function_arguments_list = function->arguments->as()->children; + for (const auto & argument : function_arguments_list) + function_node->getArguments().getNodes().push_back(getExpression(argument)); + } + + result = function_node; + } + } + else if (const auto * subquery = expression->as()) + { + auto subquery_query = subquery->children[0]; + auto query_node = getSelectWithUnionExpression(subquery_query, false); + result = query_node; + } + else if (const auto * columns_regexp_matcher = expression->as()) + { + auto column_transformers = getColumnTransformers(expression, 0 /*start_child_index*/); + result = std::make_shared(columns_regexp_matcher->getMatcher(), std::move(column_transformers)); + } + else if (const auto * columns_list_matcher = expression->as()) + { + Identifiers column_list_identifiers; + column_list_identifiers.reserve(columns_list_matcher->column_list->children.size()); + + for (auto & column_list_child : columns_list_matcher->column_list->children) + { + auto & column_list_identifier = column_list_child->as(); + column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts}); + } + + auto column_transformers = getColumnTransformers(expression, 0 /*start_child_index*/); + result = std::make_shared(std::move(column_list_identifiers), std::move(column_transformers)); + } + else if (const auto * qualified_columns_regexp_matcher = expression->as()) + { + auto & qualified_identifier = qualified_columns_regexp_matcher->children.at(0)->as(); + auto column_transformers = getColumnTransformers(expression, 1 /*start_child_index*/); + result = std::make_shared(Identifier(qualified_identifier.name_parts), qualified_columns_regexp_matcher->getMatcher(), std::move(column_transformers)); + } + else if (const auto * qualified_columns_list_matcher = expression->as()) + { + auto & qualified_identifier = qualified_columns_list_matcher->children.at(0)->as(); + + Identifiers column_list_identifiers; + column_list_identifiers.reserve(qualified_columns_list_matcher->column_list->children.size()); + + for (auto & column_list_child : qualified_columns_list_matcher->column_list->children) + { + auto & column_list_identifier = column_list_child->as(); + column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts}); + } + + auto column_transformers = getColumnTransformers(expression, 1 /*start_child_index*/); + result = std::make_shared(Identifier(qualified_identifier.name_parts), column_list_identifiers, std::move(column_transformers)); + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only literals and constants are supported as expression. Actual {}", expression->formatForErrorMessage()); + } + + result->setAlias(expression->tryGetAlias()); + result->setOriginalAST(expression); + + return result; +} + +QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_query) const +{ + if (!tables_in_select_query) + { + /** If no table is specified in SELECT query we substitude system.one table. + * SELECT * FROM system.one; + */ + Identifier storage_identifier("system.one"); + auto table = resolveStorage(storage_identifier); + return std::make_shared(table, getContext()); + } + + auto & tables = tables_in_select_query->as(); + if (tables.children.size() > 1) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Multiple tables are not supported"); + + for (const auto & table_element_untyped : tables.children) + { + const auto & table_element = table_element_untyped->as(); + + if (table_element.table_expression) + { + auto * table_expression = table_element.table_expression->as(); + + if (table_expression->database_and_table_name) + { + /// Add CTE support + auto & table_identifier_typed = table_expression->database_and_table_name->as(); + auto storage_identifier = Identifier(table_identifier_typed.name_parts); + auto table = resolveStorage(storage_identifier); + auto node = std::make_shared(table, getContext()); + + node->setAlias(table_identifier_typed.tryGetAlias()); + node->setOriginalAST(table_element.table_expression); + + return node; + } + // else if (auto * subquery_expression = table_expression->subquery->as()) + // { + // const auto & select_with_union_query = subquery_expression->children[0]; + // auto expression = getSelectWithUnionExpression(select_with_union_query, false /*scalar query*/); + // expression->setAlias(subquery_expression->tryGetAlias()); + + // if (auto * select_expression = expression->as()) + // scope->addInnerScope(select_expression->getScope()); + + // table_expressions.emplace_back(std::move(expression)); + // } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only table is supported"); + } + } + + // if (table_element.table_join) + // { + // const auto & table_join = table_element.table_join->as(); + + // auto right_table_expression = std::move(table_expressions.back()); + // table_expressions.pop_back(); + + // auto left_table_expression = std::move(table_expressions.back()); + // table_expressions.pop_back(); + + // auto join_expression = JoinExpression::create(); + // join_expression->getLeftTableExpression() = left_table_expression; + // join_expression->getRightTableExpression() = right_table_expression; + + // if (table_join->using_expression_list) + // join_expression->getUsingExpressions() = getExpressionElements(table_join->using_expression_list, scope); + + // if (table_join->on_expression) + // { + // join_expression->getOnExpression() = getExpressionElement(table_join->on_expression, scope); + // } + + // table_expressions.emplace_back(std::move(join_expression)); + // } + + // if (table_element.array_join) + // { + // auto array_join_array_expression = table_element.array_join->children[0]->children[0]; + // auto expression_element = getExpressionElement(array_join_array_expression, scope); + // expression_element->setAlias(array_join_array_expression->tryGetAlias()); + + // auto last_table_expression = std::move(table_expressions.back()); + // table_expressions.pop_back(); + + // auto array_join_expression = ArrayJoinExpression::create(); + // array_join_expression->getLeftTableExpression() = std::move(last_table_expression); + // array_join_expression->getArrayExpression() = std::move(expression_element); + + // table_expressions.push_back(array_join_expression); + // } + } + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "From section is unsupported"); + // if (table_expressions.empty()) + // throw Exception(ErrorCodes::LOGICAL_ERROR, "QueryAnalyzer from cannot be empty"); + + // return table_expressions.back(); +} + + +ColumnTransformersNodes QueryTreeBuilder::getColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const +{ + ColumnTransformersNodes column_transformers; + size_t children_size = matcher_expression->children.size(); + + for (; start_child_index < children_size; ++start_child_index) + { + const auto & child = matcher_expression->children[start_child_index]; + + if (auto * apply_transformer = child->as()) + { + if (apply_transformer->lambda) + { + auto lambda_query_tree_node = getExpression(apply_transformer->lambda); + column_transformers.emplace_back(std::make_shared(std::move(lambda_query_tree_node))); + } + else + { + auto function_node = std::make_shared(apply_transformer->func_name); + if (apply_transformer->parameters) + function_node->getParametersNode() = getExpressionList(apply_transformer->parameters); + + column_transformers.emplace_back(std::make_shared(std::move(function_node))); + } + } + else if (auto * except_transformer = child->as()) + { + auto matcher = except_transformer->getMatcher(); + if (matcher) + { + column_transformers.emplace_back(std::make_shared(std::move(matcher))); + } + else + { + Names except_column_names; + except_column_names.reserve(except_transformer->children.size()); + + for (auto & except_transformer_child : except_transformer->children) + except_column_names.push_back(except_transformer_child->as().full_name); + + column_transformers.emplace_back(std::make_shared(std::move(except_column_names))); + } + } + else if (auto * replace_transformer = child->as()) + { + std::vector replacements; + replacements.reserve(replace_transformer->children.size()); + + for (const auto & replace_transformer_child : replace_transformer->children) + { + auto & replacement = replace_transformer_child->as(); + replacements.emplace_back(ReplaceColumnTransformerNode::Replacement{replacement.name, getExpression(replacement.expr)}); + } + + column_transformers.emplace_back(std::make_shared(replacements)); + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Unsupported column matcher {}", child->formatForErrorMessage()); + } + } + + return column_transformers; +} + +StoragePtr QueryTreeBuilder::resolveStorage(const Identifier & storage_identifier) const +{ + size_t parts_size = storage_identifier.getPartsSize(); + if (parts_size < 1 || parts_size > 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table identifier should consist of 1 or 2 parts"); + + std::string database_name; + std::string table_name; + + if (storage_identifier.isCompound()) + { + database_name = storage_identifier[0]; + table_name = storage_identifier[1]; + } + else + { + table_name = storage_identifier[0]; + } + + auto current_context = getContext(); + if (database_name.empty()) + database_name = current_context->getCurrentDatabase(); + + auto & database_catalog = DatabaseCatalog::instance(); + auto database = database_catalog.tryGetDatabase(database_name); + if (!database) + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exists", database_name); + + auto table = database->tryGetTable(table_name, current_context); + + if (!table) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exists", table_name); + + return table; +} + +QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context) +{ + QueryTreeBuilder builder(query, context); + return builder.getQueryTreeNode(); +} + +} diff --git a/src/Analyzer/QueryTreeBuilder.h b/src/Analyzer/QueryTreeBuilder.h new file mode 100644 index 00000000000..3c02527436b --- /dev/null +++ b/src/Analyzer/QueryTreeBuilder.h @@ -0,0 +1,19 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +/** Build query tree from AST. + * AST that represent query ASTSelectWithUnionQuery, ASTSelectQuery. + * AST that represent a list of expressions ASTExpressionList. + * AST that represent expression ASTIdentifier, ASTAsterisk, ASTLiteral, ASTFunction. + */ +QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context); + +} diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp new file mode 100644 index 00000000000..9333fc37cb6 --- /dev/null +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -0,0 +1,91 @@ +#include + +#include +#include + +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +QueryTreePassManager::QueryTreePassManager(ContextPtr context_) : WithContext(context_) {} + +void QueryTreePassManager::addPass(QueryTreePassPtr pass) +{ + passes.push_back(std::move(pass)); +} + +void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node) +{ + auto current_context = getContext(); + size_t optimizations_size = passes.size(); + + for (size_t i = 0; i < optimizations_size; ++i) + passes[i]->run(query_tree_node, current_context); +} + +void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node, size_t up_to_pass_index) +{ + size_t optimizations_size = passes.size(); + if (up_to_pass_index > optimizations_size) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Requested to run optimizations up to {} pass. There are only {} pass", + up_to_pass_index, + optimizations_size); + + auto current_context = getContext(); + for (size_t i = 0; i < up_to_pass_index; ++i) + passes[i]->run(query_tree_node, current_context); +} + +void QueryTreePassManager::dump(WriteBuffer & buffer) +{ + size_t passes_size = passes.size(); + + for (size_t i = 0; i < passes_size; ++i) + { + auto & pass = passes[i]; + buffer << "Pass " << (i + 1) << ' ' << pass->getName() << " - " << pass->getDescription(); + if (i < passes_size) + buffer << '\n'; + } +} + +void QueryTreePassManager::dump(WriteBuffer & buffer, size_t up_to_pass_index) +{ + size_t optimizations_size = passes.size(); + if (up_to_pass_index > optimizations_size) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Requested to dump optimizations up to {} pass. There are only {} pass", + up_to_pass_index, + optimizations_size); + + for (size_t i = 0; i < up_to_pass_index; ++i) + { + auto & pass = passes[i]; + buffer << "Pass " << (i + 1) << " " << pass->getName() << " - " << pass->getDescription(); + if (i < up_to_pass_index) + buffer << '\n'; + } +} + +void addQueryTreePasses(QueryTreePassManager & manager) +{ + auto context = manager.getContext(); + const auto & settings = context->getSettingsRef(); + + manager.addPass(std::make_shared()); + + if (settings.optimize_multiif_to_if) + manager.addPass(std::make_shared()); +} + +} diff --git a/src/Analyzer/QueryTreePassManager.h b/src/Analyzer/QueryTreePassManager.h new file mode 100644 index 00000000000..0360d0bcae9 --- /dev/null +++ b/src/Analyzer/QueryTreePassManager.h @@ -0,0 +1,49 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +/** Query tree pass manager provide functionality to register and run passes + * on query tree. + */ +class QueryTreePassManager : public WithContext +{ +public: + explicit QueryTreePassManager(ContextPtr context_); + + /// Get registered passes + const std::vector & getPasses() const + { + return passes; + } + + /// Add query tree pass + void addPass(QueryTreePassPtr pass); + + /// Run query tree passes on query tree + void run(QueryTreeNodePtr query_tree_node); + + /** Run query tree passes on query tree up to up_to_pass_index. + * If up_to_pass_index is greater than passes size throws exception. + */ + void run(QueryTreeNodePtr query_tree_node, size_t up_to_pass_index); + + /// Dump query tree passes + void dump(WriteBuffer & buffer); + + /** Dump query tree passes to up_to_pass_index. + * If up_to_pass_index is greater than passes size throws exception. + */ + void dump(WriteBuffer & buffer, size_t up_to_pass_index); + +private: + std::vector passes; +}; + +void addQueryTreePasses(QueryTreePassManager & manager); + +} diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp new file mode 100644 index 00000000000..0b7e4f83015 --- /dev/null +++ b/src/Analyzer/TableNode.cpp @@ -0,0 +1,60 @@ +#include + +#include +#include +#include + +#include + +#include + +#include + +namespace DB +{ + +TableNode::TableNode(StoragePtr storage_, ContextPtr context) + : storage(std::move(storage_)) + , storage_id(storage->getStorageID()) + , table_lock(storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)) + , storage_snapshot(storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context)) +{ +} + +void TableNode::dumpTree(WriteBuffer & buffer, size_t indent) const +{ + buffer << std::string(indent, ' ') << "TABLE "; + writePointerHex(this, buffer); + buffer << ' ' << storage_id.getFullNameNotQuoted(); +} + +void TableNode::updateTreeHashImpl(HashState & state) const +{ + auto full_name = storage_id.getFullNameNotQuoted(); + state.update(full_name.size()); + state.update(full_name); +} + +String TableNode::getName() const +{ + return storage->getStorageID().getFullNameNotQuoted(); +} + +ASTPtr TableNode::toASTImpl() const +{ + return std::make_shared(storage_id.getDatabaseName(), storage_id.getTableName()); +} + +QueryTreeNodePtr TableNode::cloneImpl() const +{ + TableNodePtr result_table_node(new TableNode()); + + result_table_node->storage = storage; + result_table_node->storage_id = storage_id; + result_table_node->table_lock = table_lock; + result_table_node->storage_snapshot = storage_snapshot; + + return result_table_node; +} + +} diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h new file mode 100644 index 00000000000..d1212dff36c --- /dev/null +++ b/src/Analyzer/TableNode.h @@ -0,0 +1,90 @@ +#pragma once + +#include +#include +#include + +#include +#include + +#include + +namespace DB +{ + +/** Table node represents table in query tree. + * Example: SELECT a FROM test_table. + * test_table - is identifier, that during query analysis pass must be resolved into table node. + * + * During construction table node: + * 1. Lock storage for share. Later lock can be moved out of node using `moveTableLock` method. + * 2. Take storage snapshot. + */ +class TableNode; +using TableNodePtr = std::shared_ptr; + +class TableNode : public IQueryTreeNode +{ +public: + /// Construct table node with storage and context + explicit TableNode(StoragePtr storage_, ContextPtr context); + + /// Get storage + const StoragePtr & getStorage() const + { + return storage; + } + + /// Get storage id + const StorageID & getStorageID() const + { + return storage_id; + } + + /// Get storage snapshot + const StorageSnapshotPtr & getStorageSnapshot() const + { + return storage_snapshot; + } + + /// Get table lock + const TableLockHolder & getTableLock() const + { + return table_lock; + } + + /** Move table lock out of table node. + * After using this method table node state becomes invalid. + */ + TableLockHolder && moveTableLock() + { + return std::move(table_lock); + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::TABLE; + } + + void dumpTree(WriteBuffer & buffer, size_t indent) const override; + + String getName() const override; + +protected: + void updateTreeHashImpl(HashState & state) const override; + + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + TableNode() : storage_id("", "") {} + + StoragePtr storage; + StorageID storage_id; + TableLockHolder table_lock; + StorageSnapshotPtr storage_snapshot; +}; + +} + diff --git a/src/Analyzer/examples/CMakeLists.txt b/src/Analyzer/examples/CMakeLists.txt new file mode 100644 index 00000000000..c6b1b0b3c5f --- /dev/null +++ b/src/Analyzer/examples/CMakeLists.txt @@ -0,0 +1,3 @@ +add_executable (query_analyzer query_analyzer.cpp) +target_include_directories (query_analyzer SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR}) +target_link_libraries (query_analyzer PRIVATE dbms) diff --git a/src/Analyzer/examples/query_analyzer.cpp b/src/Analyzer/examples/query_analyzer.cpp new file mode 100644 index 00000000000..0e9073306a0 --- /dev/null +++ b/src/Analyzer/examples/query_analyzer.cpp @@ -0,0 +1,11 @@ +#include + +using namespace DB; + +int main(int argc, char ** argv) +{ + (void)(argc); + (void)(argv); + + return 0; +} diff --git a/src/Analyzer/tests/CMakeLists.txt b/src/Analyzer/tests/CMakeLists.txt new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Analyzer/tests/gtest_identifier.cpp b/src/Analyzer/tests/gtest_identifier.cpp new file mode 100644 index 00000000000..4f224dfbb07 --- /dev/null +++ b/src/Analyzer/tests/gtest_identifier.cpp @@ -0,0 +1,227 @@ +#include + +#include + +using namespace DB; + +TEST(Identifier, IdentifierBasics) +{ + { + Identifier identifier; + + ASSERT_TRUE(identifier.empty()); + ASSERT_TRUE(identifier.isEmpty()); + ASSERT_EQ(identifier.getPartsSize(), 0); + ASSERT_FALSE(identifier.isShort()); + ASSERT_FALSE(identifier.isCompound()); + ASSERT_FALSE(identifier.startsWith("test")); + ASSERT_FALSE(identifier.endsWith("test")); + ASSERT_EQ(identifier.begin(), identifier.end()); + ASSERT_EQ(identifier.getFullName(), ""); + } + { + Identifier identifier("value"); + + ASSERT_FALSE(identifier.empty()); + ASSERT_FALSE(identifier.isEmpty()); + ASSERT_EQ(identifier.getPartsSize(), 1); + ASSERT_TRUE(identifier.isShort()); + ASSERT_FALSE(identifier.isCompound()); + ASSERT_EQ(identifier.front(), "value"); + ASSERT_EQ(identifier.back(), "value"); + ASSERT_FALSE(identifier.startsWith("test")); + ASSERT_FALSE(identifier.endsWith("test")); + ASSERT_TRUE(identifier.startsWith("value")); + ASSERT_TRUE(identifier.endsWith("value")); + ASSERT_EQ(identifier[0], "value"); + ASSERT_NE(identifier.begin(), identifier.end()); + ASSERT_EQ(identifier.getFullName(), "value"); + } + { + Identifier identifier("value1.value2"); + + ASSERT_FALSE(identifier.empty()); + ASSERT_FALSE(identifier.isEmpty()); + ASSERT_EQ(identifier.getPartsSize(), 2); + ASSERT_FALSE(identifier.isShort()); + ASSERT_TRUE(identifier.isCompound()); + ASSERT_EQ(identifier.front(), "value1"); + ASSERT_EQ(identifier.back(), "value2"); + ASSERT_FALSE(identifier.startsWith("test")); + ASSERT_FALSE(identifier.endsWith("test")); + ASSERT_TRUE(identifier.startsWith("value1")); + ASSERT_TRUE(identifier.endsWith("value2")); + ASSERT_EQ(identifier[0], "value1"); + ASSERT_EQ(identifier[1], "value2"); + ASSERT_NE(identifier.begin(), identifier.end()); + ASSERT_EQ(identifier.getFullName(), "value1.value2"); + } + { + Identifier identifier1("value1.value2"); + Identifier identifier2("value1.value2"); + + ASSERT_EQ(identifier1, identifier2); + } + { + Identifier identifier1("value1.value2"); + Identifier identifier2("value1.value3"); + + ASSERT_NE(identifier1, identifier2); + } +} + +TEST(Identifier, IdentifierPopParts) +{ + { + Identifier identifier("value1.value2.value3"); + + ASSERT_EQ(identifier.getFullName(), "value1.value2.value3"); + identifier.popLast(); + ASSERT_EQ(identifier.getFullName(), "value1.value2"); + identifier.popLast(); + ASSERT_EQ(identifier.getFullName(), "value1"); + identifier.popLast(); + ASSERT_EQ(identifier.getFullName(), ""); + ASSERT_TRUE(identifier.isEmpty()); + } + { + Identifier identifier("value1.value2.value3"); + + ASSERT_EQ(identifier.getFullName(), "value1.value2.value3"); + identifier.popFirst(); + ASSERT_EQ(identifier.getFullName(), "value2.value3"); + identifier.popFirst(); + ASSERT_EQ(identifier.getFullName(), "value3"); + identifier.popFirst(); + ASSERT_EQ(identifier.getFullName(), ""); + ASSERT_TRUE(identifier.isEmpty()); + } + { + Identifier identifier("value1.value2.value3"); + + ASSERT_EQ(identifier.getFullName(), "value1.value2.value3"); + identifier.popLast(); + ASSERT_EQ(identifier.getFullName(), "value1.value2"); + identifier.popFirst(); + ASSERT_EQ(identifier.getFullName(), "value2"); + identifier.popLast(); + ASSERT_EQ(identifier.getFullName(), ""); + ASSERT_TRUE(identifier.isEmpty()); + } +} + +TEST(Identifier, IdentifierViewBasics) +{ + { + Identifier identifier; + IdentifierView identifier_view(identifier); + + ASSERT_TRUE(identifier_view.empty()); + ASSERT_TRUE(identifier_view.isEmpty()); + ASSERT_EQ(identifier_view.getPartsSize(), 0); + ASSERT_FALSE(identifier_view.isShort()); + ASSERT_FALSE(identifier_view.isCompound()); + ASSERT_FALSE(identifier_view.startsWith("test")); + ASSERT_FALSE(identifier_view.endsWith("test")); + ASSERT_EQ(identifier_view.begin(), identifier_view.end()); + ASSERT_EQ(identifier_view.getFullName(), ""); + } + { + Identifier identifier("value"); + IdentifierView identifier_view(identifier); + + ASSERT_FALSE(identifier_view.empty()); + ASSERT_FALSE(identifier_view.isEmpty()); + ASSERT_EQ(identifier_view.getPartsSize(), 1); + ASSERT_TRUE(identifier_view.isShort()); + ASSERT_FALSE(identifier_view.isCompound()); + ASSERT_EQ(identifier_view.front(), "value"); + ASSERT_EQ(identifier_view.back(), "value"); + ASSERT_FALSE(identifier_view.startsWith("test")); + ASSERT_FALSE(identifier_view.endsWith("test")); + ASSERT_TRUE(identifier_view.startsWith("value")); + ASSERT_TRUE(identifier_view.endsWith("value")); + ASSERT_EQ(identifier_view[0], "value"); + ASSERT_NE(identifier_view.begin(), identifier_view.end()); + ASSERT_EQ(identifier_view.getFullName(), "value"); + } + { + Identifier identifier("value1.value2"); + IdentifierView identifier_view(identifier); + + ASSERT_FALSE(identifier_view.empty()); + ASSERT_FALSE(identifier_view.isEmpty()); + ASSERT_EQ(identifier_view.getPartsSize(), 2); + ASSERT_FALSE(identifier_view.isShort()); + ASSERT_TRUE(identifier_view.isCompound()); + ASSERT_FALSE(identifier_view.startsWith("test")); + ASSERT_FALSE(identifier_view.endsWith("test")); + ASSERT_TRUE(identifier_view.startsWith("value1")); + ASSERT_TRUE(identifier_view.endsWith("value2")); + ASSERT_EQ(identifier_view[0], "value1"); + ASSERT_EQ(identifier_view[1], "value2"); + ASSERT_NE(identifier_view.begin(), identifier_view.end()); + ASSERT_EQ(identifier_view.getFullName(), "value1.value2"); + } + { + Identifier identifier1("value1.value2"); + IdentifierView identifier_view1(identifier1); + + Identifier identifier2("value1.value2"); + IdentifierView identifier_view2(identifier2); + + ASSERT_EQ(identifier_view1, identifier_view2); + } + { + Identifier identifier1("value1.value2"); + IdentifierView identifier_view1(identifier1); + + Identifier identifier2("value1.value3"); + IdentifierView identifier_view2(identifier2); + + ASSERT_NE(identifier_view1, identifier_view2); + } +} + +TEST(Identifier, IdentifierViewPopParts) +{ + { + Identifier identifier("value1.value2.value3"); + IdentifierView identifier_view(identifier); + + ASSERT_EQ(identifier_view.getFullName(), "value1.value2.value3"); + identifier_view.popLast(); + ASSERT_EQ(identifier_view.getFullName(), "value1.value2"); + identifier_view.popLast(); + ASSERT_EQ(identifier_view.getFullName(), "value1"); + identifier_view.popLast(); + ASSERT_EQ(identifier_view.getFullName(), ""); + ASSERT_TRUE(identifier_view.isEmpty()); + } + { + Identifier identifier("value1.value2.value3"); + IdentifierView identifier_view(identifier); + + ASSERT_EQ(identifier_view.getFullName(), "value1.value2.value3"); + identifier_view.popFirst(); + ASSERT_EQ(identifier_view.getFullName(), "value2.value3"); + identifier_view.popFirst(); + ASSERT_EQ(identifier_view.getFullName(), "value3"); + identifier_view.popFirst(); + ASSERT_EQ(identifier_view.getFullName(), ""); + ASSERT_TRUE(identifier_view.isEmpty()); + } + { + Identifier identifier("value1.value2.value3"); + IdentifierView identifier_view(identifier); + + ASSERT_EQ(identifier_view.getFullName(), "value1.value2.value3"); + identifier_view.popLast(); + ASSERT_EQ(identifier_view.getFullName(), "value1.value2"); + identifier_view.popFirst(); + ASSERT_EQ(identifier_view.getFullName(), "value2"); + identifier_view.popLast(); + ASSERT_EQ(identifier_view.getFullName(), ""); + ASSERT_TRUE(identifier_view.isEmpty()); + } +} diff --git a/src/Analyzer/tests/gtest_query_tree_node.cpp b/src/Analyzer/tests/gtest_query_tree_node.cpp new file mode 100644 index 00000000000..95317273fd3 --- /dev/null +++ b/src/Analyzer/tests/gtest_query_tree_node.cpp @@ -0,0 +1,82 @@ +#include + +#include + +#include +#include +#include + +using namespace DB; + +class SourceNode final : public IQueryTreeNode +{ +public: + void updateTreeHashImpl(HashState & hash_state) const override + { + (void)(hash_state); + } + + void dumpTree(WriteBuffer & buffer, size_t indent) const override + { + (void)(buffer); + (void)(indent); + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::TABLE; + } + + ASTPtr toASTImpl() const override + { + return nullptr; + } + + QueryTreeNodePtr cloneImpl() const override + { + return std::make_shared(); + } +}; + +TEST(QueryTreeNode, Clone) +{ + { + auto source_node = std::make_shared(); + + NameAndTypePair column_name_and_type("value", std::make_shared()); + auto column_node = std::make_shared(column_name_and_type, source_node); + + ASSERT_EQ(column_node->getColumnSource().get(), source_node.get()); + + auto cloned_column_node = column_node->clone(); + + /// If in subtree source was not cloned, source pointer must remain same + ASSERT_NE(column_node.get(), cloned_column_node.get()); + ASSERT_EQ(cloned_column_node->as().getColumnSource().get(), source_node.get()); + } + { + auto root_node = std::make_shared(); + auto source_node = std::make_shared(); + + NameAndTypePair column_name_and_type("value", std::make_shared()); + auto column_node = std::make_shared(column_name_and_type, source_node); + + root_node->getNodes().push_back(source_node); + root_node->getNodes().push_back(column_node); + + ASSERT_EQ(column_node->getColumnSource().get(), source_node.get()); + + auto cloned_root_node = std::static_pointer_cast(root_node->clone()); + auto cloned_source_node = cloned_root_node->getNodes()[0]; + auto cloned_column_node = std::static_pointer_cast(cloned_root_node->getNodes()[1]); + + /** If in subtree source was cloned. + * Source pointer for node that was cloned must remain same. + * Source pointer for cloned node must be updated. + */ + ASSERT_NE(column_node.get(), cloned_column_node.get()); + ASSERT_NE(source_node.get(), cloned_source_node.get()); + ASSERT_EQ(column_node->getColumnSource().get(), source_node.get()); + ASSERT_EQ(cloned_column_node->getColumnSource().get(), cloned_source_node.get()); + } +} diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 419a34fc2ab..aa681126c73 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -66,6 +66,7 @@ add_subdirectory (Storages) add_subdirectory (Parsers) add_subdirectory (IO) add_subdirectory (Functions) +add_subdirectory (Analyzer) add_subdirectory (Interpreters) add_subdirectory (AggregateFunctions) add_subdirectory (Client) @@ -254,6 +255,7 @@ add_object_library(clickhouse_datatypes_serializations DataTypes/Serializations) add_object_library(clickhouse_databases Databases) add_object_library(clickhouse_databases_mysql Databases/MySQL) add_object_library(clickhouse_disks Disks) +add_object_library(clickhouse_analyzer Analyzer) add_object_library(clickhouse_interpreters Interpreters) add_object_library(clickhouse_interpreters_cache Interpreters/Cache) add_object_library(clickhouse_interpreters_access Interpreters/Access) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0b8d24b1abc..bf117163431 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -302,6 +302,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Float, opentelemetry_start_trace_probability, 0., "Probability to start an OpenTelemetry trace for an incoming query.", 0) \ M(Bool, opentelemetry_trace_processors, false, "Collect OpenTelemetry spans for processors.", 0) \ M(Bool, prefer_column_name_to_alias, false, "Prefer using column names instead of aliases if possible.", 0) \ + M(Bool, use_analyzer, false, "Use analyzer", 0) \ M(Bool, prefer_global_in_and_join, false, "If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.", 0) \ \ \ diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 6d500cc15c4..dfed7cedcf0 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -185,8 +185,10 @@ public: const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); if (!data_type_function) - throw Exception("First argument for function " + getName() + " must be a function", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be a function. Actual {}", + getName(), + arguments[0].type->getName()); /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 4f7ddda6b0b..6ac36dc80ed 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -82,7 +82,10 @@ public: const DataTypeTuple * tuple = checkAndGetDataType(tuple_col); if (!tuple) - throw Exception("First argument for function " + getName() + " must be tuple or array of tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be tuple or array of tuple. Actual {}", + getName(), + arguments[0].type->getName()); auto index = getElementNum(arguments[1].column, *tuple, number_of_arguments); if (index.has_value()) @@ -137,7 +140,10 @@ public: const DataTypeTuple * tuple_type_concrete = checkAndGetDataType(tuple_type); const ColumnTuple * tuple_col_concrete = checkAndGetColumn(tuple_col); if (!tuple_type_concrete || !tuple_col_concrete) - throw Exception("First argument for function " + getName() + " must be tuple or array of tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be tuple or array of tuple. Actual {}", + getName(), + first_arg.type->getName()); auto index = getElementNum(arguments[1].column, *tuple_type_concrete, arguments.size()); @@ -221,20 +227,18 @@ private: std::optional getElementNum(const ColumnPtr & index_column, const DataTypeTuple & tuple, const size_t argument_size) const { - if ( - checkAndGetColumnConst(index_column.get()) - || checkAndGetColumnConst(index_column.get()) - || checkAndGetColumnConst(index_column.get()) - || checkAndGetColumnConst(index_column.get()) - ) + if (checkAndGetColumnConst(index_column.get()) + || checkAndGetColumnConst(index_column.get()) + || checkAndGetColumnConst(index_column.get()) + || checkAndGetColumnConst(index_column.get())) { size_t index = index_column->getUInt(0); if (index == 0) - throw Exception("Indices in tuples are 1-based.", ErrorCodes::ILLEGAL_INDEX); + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indices in tuples are 1-based."); if (index > tuple.getElements().size()) - throw Exception("Index for tuple element is out of range.", ErrorCodes::ILLEGAL_INDEX); + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Index for tuple element is out of range."); return std::optional(index - 1); } @@ -253,7 +257,9 @@ private: return std::nullopt; } else - throw Exception("Second argument to " + getName() + " must be a constant UInt or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Second argument to {} must be a constant UInt or String", + getName()); } }; diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 0524feea1f6..e0be9ba6109 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -17,7 +18,6 @@ #include #include - namespace DB { @@ -60,7 +60,6 @@ Block InterpreterDescribeQuery::getSampleBlock(bool include_subcolumns) return block; } - BlockIO InterpreterDescribeQuery::execute() { ColumnsDescription columns; @@ -72,8 +71,20 @@ BlockIO InterpreterDescribeQuery::execute() if (table_expression.subquery) { - auto names_and_types = InterpreterSelectWithUnionQuery::getSampleBlock( - table_expression.subquery->children.at(0), getContext()).getNamesAndTypesList(); + NamesAndTypesList names_and_types; + auto select_query = table_expression.subquery->children.at(0); + auto current_context = getContext(); + + if (settings.use_analyzer) + { + SelectQueryOptions select_query_options; + names_and_types = InterpreterSelectQueryAnalyzer(select_query, select_query_options, current_context).getSampleBlock().getNamesAndTypesList(); + } + else + { + names_and_types = InterpreterSelectWithUnionQuery::getSampleBlock(select_query, current_context).getNamesAndTypesList(); + } + columns = ColumnsDescription(std::move(names_and_types)); } else if (table_expression.table_function) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 4799970b6a1..dbc742c9cb3 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -28,6 +28,9 @@ #include +#include +#include + namespace DB { @@ -155,6 +158,28 @@ struct QueryASTSettings {"graph", graph}, {"optimize", optimize} }; + + std::unordered_map> integer_settings; +}; + +struct QueryTreeSettings +{ + bool run_passes = false; + Int64 passes = -1; + bool ast = false; + + constexpr static char name[] = "QUERYTREE"; + + std::unordered_map> boolean_settings = + { + {"run_passes", run_passes}, + {"ast", ast} + }; + + std::unordered_map> integer_settings = + { + {"passes", passes} + }; }; struct QueryPlanSettings @@ -177,6 +202,8 @@ struct QueryPlanSettings {"json", json}, {"sorting", query_plan_options.sorting}, }; + + std::unordered_map> integer_settings; }; struct QueryPipelineSettings @@ -193,18 +220,31 @@ struct QueryPipelineSettings {"graph", graph}, {"compact", compact}, }; + + std::unordered_map> integer_settings; }; template struct ExplainSettings : public Settings { using Settings::boolean_settings; + using Settings::integer_settings; bool has(const std::string & name_) const + { + return hasBooleanSetting(name_) || hasIntegerSetting(name_); + } + + bool hasBooleanSetting(const std::string & name_) const { return boolean_settings.count(name_) > 0; } + bool hasIntegerSetting(const std::string & name_) const + { + return integer_settings.count(name_) > 0; + } + void setBooleanSetting(const std::string & name_, bool value) { auto it = boolean_settings.find(name_); @@ -214,6 +254,15 @@ struct ExplainSettings : public Settings it->second.get() = value; } + void setIntegerSetting(const std::string & name_, Int64 value) + { + auto it = integer_settings.find(name_); + if (it == integer_settings.end()) + throw Exception("Unknown setting for ExplainSettings: " + name_, ErrorCodes::LOGICAL_ERROR); + + it->second.get() = value; + } + std::string getSettingsList() const { std::string res; @@ -224,6 +273,13 @@ struct ExplainSettings : public Settings res += setting.first; } + for (const auto & setting : integer_settings) + { + if (!res.empty()) + res += ", "; + + res += setting.first; + } return res; } @@ -246,15 +302,23 @@ ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) if (change.value.getType() != Field::Types::UInt64) throw Exception(ErrorCodes::INVALID_SETTING_VALUE, - "Invalid type {} for setting \"{}\" only boolean settings are supported", + "Invalid type {} for setting \"{}\" only integer settings are supported", change.value.getTypeName(), change.name); - auto value = change.value.get(); - if (value > 1) - throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.name + - "\". Only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE); + if (settings.hasBooleanSetting(change.name)) + { + auto value = change.value.get(); + if (value > 1) + throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.name + + "\". Expected boolean type", ErrorCodes::INVALID_SETTING_VALUE); - settings.setBooleanSetting(change.name, value); + settings.setBooleanSetting(change.name, value); + } + else + { + auto value = change.value.get(); + settings.setIntegerSetting(change.name, value); + } } return settings; @@ -304,6 +368,50 @@ QueryPipeline InterpreterExplainQuery::executeImpl() ast.getExplainedQuery()->format(IAST::FormatSettings(buf, false)); break; } + case ASTExplainQuery::QueryTree: + { + if (ast.getExplainedQuery()->as() == nullptr) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT is supported for EXPLAIN QUERYTREE query"); + + auto settings = checkAndGetSettings(ast.getSettings()); + auto query_tree = buildQueryTree(ast.getExplainedQuery(), getContext()); + + if (settings.run_passes) + { + auto query_tree_pass_manager = QueryTreePassManager(getContext()); + addQueryTreePasses(query_tree_pass_manager); + + if (settings.passes < 0) + { + query_tree_pass_manager.dump(buf); + buf << '\n'; + query_tree_pass_manager.run(query_tree); + } + else + { + size_t pass_index = static_cast(settings.passes); + query_tree_pass_manager.dump(buf, pass_index); + if (pass_index > 0) + buf << '\n'; + + query_tree_pass_manager.run(query_tree, pass_index); + } + + query_tree->dumpTree(buf, 0); + } + else + { + query_tree->dumpTree(buf, 0); + } + + if (settings.ast) + { + buf << '\n'; + query_tree->toAST()->format(IAST::FormatSettings(buf, false)); + } + + break; + } case ASTExplainQuery::QueryPlan: { if (!dynamic_cast(ast.getExplainedQuery().get())) diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index ca0a59c0c1a..3e074ec8713 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -63,6 +63,7 @@ #include #include #include +#include #include #include #include @@ -118,6 +119,9 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut if (query->as()) { + if (context->getSettingsRef().use_analyzer) + return std::make_unique(query, options, context); + /// This is internal part of ASTSelectWithUnionQuery. /// Even if there is SELECT without union, it is represented by ASTSelectWithUnionQuery with single ASTSelectQuery as a child. return std::make_unique(query, context, options); @@ -125,6 +129,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut else if (query->as()) { ProfileEvents::increment(ProfileEvents::SelectQuery); + + if (context->getSettingsRef().use_analyzer) + return std::make_unique(query, options, context); + return std::make_unique(query, context, options); } else if (query->as()) diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp new file mode 100644 index 00000000000..73da6ea827e --- /dev/null +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -0,0 +1,609 @@ +#include + +#include +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; + extern const int LOGICAL_ERROR; +} + +QueryPipeline buildDummyPipeline() +{ + ColumnsWithTypeAndName columns; + auto string_data_type = std::make_shared(); + + auto string_column = string_data_type->createColumn(); + string_column->insert("TestValue"); + columns.emplace_back(ColumnWithTypeAndName{std::move(string_column), string_data_type, "test_column"}); + + Block block(columns); + auto source = std::make_shared(block); + auto shell_input_pipe = Pipe(std::move(source)); + + QueryPipeline pipeline(std::move(shell_input_pipe)); + return pipeline; +} + +String dumpQueryPlan(QueryPlan & query_plan) +{ + WriteBufferFromOwnString query_plan_buffer; + query_plan.explainPlan(query_plan_buffer, QueryPlan::ExplainPlanOptions{}); + return query_plan_buffer.str(); +} + +String dumpQueryPipeline(QueryPlan & query_plan) +{ + QueryPlan::ExplainPipelineOptions explain_pipeline; + WriteBufferFromOwnString query_pipeline_buffer; + query_plan.explainPipeline(query_pipeline_buffer, explain_pipeline); + return query_pipeline_buffer.str(); +} + + +struct QueryTreeActionsScopeNode +{ + explicit QueryTreeActionsScopeNode(ActionsDAGPtr actions_dag_, QueryTreeNodePtr scope_node_) + : actions_dag(std::move(actions_dag_)) + , scope_node(std::move(scope_node_)) + { + for (const auto & node : actions_dag->getNodes()) + node_name_to_node[node.result_name] = &node; + } + + bool containsNode(const std::string & node_name) + { + return node_name_to_node.find(node_name) != node_name_to_node.end(); + } + + const ActionsDAG::Node * tryGetNode(const std::string & node_name) + { + auto it = node_name_to_node.find(node_name); + if (it == node_name_to_node.end()) + return {}; + + return it->second; + } + + const ActionsDAG::Node * getNodeOrThrow(const std::string & node_name) + { + auto it = node_name_to_node.find(node_name); + if (it == node_name_to_node.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "No node with name {}. There are only nodes {}", + node_name, + actions_dag->dumpNames()); + + return it->second; + } + + const ActionsDAG::Node * addInputColumnIfNecessary(const std::string & node_name, const DataTypePtr & column_type) + { + auto it = node_name_to_node.find(node_name); + if (it != node_name_to_node.end()) + return it->second; + + const auto * node = &actions_dag->addInput(node_name, column_type); + + // std::cout << "QueryTreeActionsScopeNode::addInputColumnIfNecessary dag " << actions_dag << " node name " << node_name; + // std::cout << " result node ptr " << node << std::endl; + + node_name_to_node[node->result_name] = node; + + return node; + } + + const ActionsDAG::Node * addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) + { + auto it = node_name_to_node.find(node_name); + if (it != node_name_to_node.end()) + return it->second; + + const auto * node = &actions_dag->addColumn(column); + + // std::cout << "QueryTreeActionsScopeNode::addConstantIfNecessary dag " << actions_dag << " node name " << node_name; + // std::cout << " result node ptr " << node << std::endl; + + node_name_to_node[node->result_name] = node; + + return node; + } + + const ActionsDAG::Node * addFunctionIfNecessary(const std::string & node_name, ActionsDAG::NodeRawConstPtrs children, FunctionOverloadResolverPtr function) + { + auto it = node_name_to_node.find(node_name); + if (it != node_name_to_node.end()) + return it->second; + + const auto * node = &actions_dag->addFunction(function, children, node_name); + + // std::cout << "QueryTreeActionsScopeNode::addFunctionIfNecessary dag " << actions_dag << " node name " << node_name; + // std::cout << " result node ptr " << node << std::endl; + + node_name_to_node[node->result_name] = node; + + return node; + } + + std::unordered_map node_name_to_node; + ActionsDAGPtr actions_dag; + QueryTreeNodePtr scope_node; +}; + +class QueryTreeActionsVisitor : public WithContext +{ +public: + explicit QueryTreeActionsVisitor( + ActionsDAGPtr actions_dag, + ContextPtr context_) + : WithContext(context_) + { + actions_stack.emplace_back(std::move(actions_dag), nullptr); + } + + ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node) + { + ActionsDAG::NodeRawConstPtrs result; + + if (auto * expression_list_node = expression_node->as()) + { + for (auto & node : expression_list_node->getNodes()) + { + auto [node_name, _] = visitImpl(node); + result.push_back(actions_stack.front().getNodeOrThrow(node_name)); + } + } + else + { + auto [node_name, _] = visitImpl(expression_node); + result.push_back(actions_stack.front().getNodeOrThrow(node_name)); + } + + return result; + } + +private: + + std::pair visitImpl(QueryTreeNodePtr node) + { + if (auto * column_node = node->as()) + return visitColumn(*column_node); + else if (auto * constant_node = node->as()) + return visitConstant(*constant_node); + else if (auto * function_node = node->as()) + return visitFunction(*function_node); + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected only column, constant or function node. Actual {}", node->formatASTForErrorMessage()); + } + + std::pair visitColumn(ColumnNode & column) + { + const auto & column_name = column.getColumnName(); + + Int64 actions_stack_size = static_cast(actions_stack.size() - 1); + for (Int64 i = actions_stack_size; i >= 0; --i) + { + actions_stack[i].addInputColumnIfNecessary(column_name, column.getColumnType()); + + if (column.getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA && + actions_stack[i].scope_node.get() == column.getColumnSource().get()) + { + return {column_name, i}; + } + } + + return {column_name, 0}; + } + + std::pair visitConstant(ConstantNode & constant_node) + { + const auto & literal = constant_node.getConstantValue(); + + auto constant_name = constant_node.getName(); + + ColumnWithTypeAndName column; + column.name = constant_name; + column.type = constant_node.getResultType(); + column.column = column.type->createColumnConst(1, literal); + + actions_stack[0].addConstantIfNecessary(constant_name, column); + + size_t actions_stack_size = actions_stack.size(); + for (size_t i = 1; i < actions_stack_size; ++i) + { + auto & actions_stack_node = actions_stack[i]; + actions_stack_node.addInputColumnIfNecessary(constant_name, column.type); + } + + return {constant_name, 0}; + } + + std::pair visitLambda(QueryTreeNodePtr lambda_node_untyped) + { + auto & lambda_node = lambda_node_untyped->as(); + auto result_type = lambda_node.getResultType(); + if (!result_type) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Lambda {} is not resolved during query analysis", + lambda_node.formatASTForErrorMessage()); + + // std::cout << "QueryTreeActionsVisitor::visitLambda " << lambda_node.formatASTForErrorMessage() << std::endl; + // std::cout << "Lambda arguments nodes size " << lambda_node.getArguments().getNodes().size() << std::endl; + + NamesAndTypesList lambda_arguments_names_and_types; + + for (auto & lambda_node_argument : lambda_node.getArguments().getNodes()) + { + auto lambda_argument_name = lambda_node_argument->getName(); + auto lambda_argument_type = lambda_node_argument->getResultType(); + // std::cout << "Lambda argument name " << lambda_argument_name; + // std::cout << " type " << lambda_argument_type->getName() << std::endl; + lambda_arguments_names_and_types.emplace_back(lambda_argument_name, lambda_argument_type); + } + + size_t previous_scope_node_actions_stack_index = actions_stack.size() - 1; + + auto lambda_actions_dag = std::make_shared(); + actions_stack.emplace_back(lambda_actions_dag, lambda_node_untyped); + + auto [node_name, level] = visitImpl(lambda_node.getExpression()); + auto lambda_result_node_name = node_name; + lambda_actions_dag->getIndex().push_back(actions_stack.back().getNodeOrThrow(node_name)); + + // std::cout << "Previous DAG nodes " << actions_stack[previous_scope_node_actions_stack_index].actions_dag.get() << std::endl; + // for (const auto & previous_actions_node : actions_stack[previous_scope_node_actions_stack_index].actions_dag->getNodes()) + // { + // std::cout << "Node " << &previous_actions_node << " result name " << previous_actions_node.result_name << std::endl; + // std::cout << "Children " << previous_actions_node.children.size() << std::endl; + // for (const auto * child_node : previous_actions_node.children) + // { + // std::cout << "Child node " << child_node << " result name " << child_node->result_name << std::endl; + // } + // } + + lambda_actions_dag->removeUnusedActions(Names(1, lambda_result_node_name)); + + // std::cout << "Lambda actions DAG Node " << node_name << " level " << level << std::endl; + // std::cout << "Lambda actions DAG " << lambda_actions_dag.get() << std::endl; + // std::cout << lambda_actions_dag->dumpDAG() << std::endl; + + auto lambda_actions = std::make_shared( + lambda_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + + Names captured_column_names; + ActionsDAG::NodeRawConstPtrs lambda_children; + Names required_column_names = lambda_actions->getRequiredColumns(); + + const auto & lambda_argument_names = lambda_node.getArgumentNames(); + + for (const auto & required_column_name : required_column_names) + { + auto it = std::find_if( + lambda_argument_names.begin(), lambda_argument_names.end(), [&](auto & value) { return value == required_column_name; }); + + if (it == lambda_argument_names.end()) + { + lambda_children.push_back(actions_stack[previous_scope_node_actions_stack_index].getNodeOrThrow(required_column_name)); + captured_column_names.push_back(required_column_name); + } + } + + auto lambda_hash = lambda_node.getTreeHash(); + std::string lambda_name = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); + + auto function_capture = std::make_shared( + lambda_actions, captured_column_names, lambda_arguments_names_and_types, result_type, lambda_result_node_name); + actions_stack.pop_back(); + + if (level == actions_stack.size()) + --level; + + actions_stack[level].addFunctionIfNecessary(lambda_name, lambda_children, function_capture); + + size_t actions_stack_size = actions_stack.size(); + for (size_t i = level + 1; i < actions_stack_size; ++i) + { + auto & actions_stack_node = actions_stack[i]; + actions_stack_node.addInputColumnIfNecessary(lambda_name, result_type); + } + + return {lambda_name, level}; + } + + std::pair visitFunction(FunctionNode & function_node) + { + auto function_node_name = function_node.getName(); + + if (function_node.isAggregateFunction()) + { + size_t actions_stack_size = actions_stack.size(); + + for (size_t i = 0; i < actions_stack_size; ++i) + { + auto & actions_stack_node = actions_stack[i]; + actions_stack_node.addInputColumnIfNecessary(function_node_name, function_node.getResultType()); + } + + return {function_node_name, 0}; + } + + auto & function_arguments = function_node.getArguments().getNodes(); + size_t function_arguments_size = function_arguments.size(); + + Names function_arguments_node_names; + function_arguments_node_names.reserve(function_arguments_size); + + size_t level = 0; + for (auto & argument : function_arguments) + { + if (argument->getNodeType() == QueryTreeNodeType::LAMBDA) + { + auto [node_name, node_min_level] = visitLambda(argument); + function_arguments_node_names.push_back(std::move(node_name)); + level = std::max(level, node_min_level); + continue; + } + + auto [node_name, node_min_level] = visitImpl(argument); + function_arguments_node_names.push_back(std::move(node_name)); + level = std::max(level, node_min_level); + } + + ActionsDAG::NodeRawConstPtrs children; + children.reserve(function_arguments_size); + + for (auto & function_argument_node_name : function_arguments_node_names) + children.push_back(actions_stack[level].getNodeOrThrow(function_argument_node_name)); + + actions_stack[level].addFunctionIfNecessary(function_node_name, children, function_node.getFunction()); + + size_t actions_stack_size = actions_stack.size(); + for (size_t i = level + 1; i < actions_stack_size; ++i) + { + auto & actions_stack_node = actions_stack[i]; + actions_stack_node.addInputColumnIfNecessary(function_node_name, function_node.getResultType()); + } + + return {function_node_name, level}; + } + + std::vector actions_stack; +}; + +class CollectSourceColumnsMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + NameSet source_columns_set; + }; + + static void visit(QueryTreeNodePtr & node, Data & data) + { + auto * column_node = node->as(); + if (!column_node) + return; + + auto * table_node = column_node->getColumnSource()->as(); + if (!table_node) + return; + + data.source_columns_set.insert(column_node->getColumnName()); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) + { + return true; + } +}; + + +using CollectSourceColumnsVisitor = CollectSourceColumnsMatcher::Visitor; + +InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer(const ASTPtr & query_ptr_, + const SelectQueryOptions & select_query_options_, + ContextPtr context_) + : WithContext(context_) + , query_ptr(query_ptr_) + , select_query_options(select_query_options_) + , query_tree_pass_manager(context_) +{ + addQueryTreePasses(query_tree_pass_manager); + + if (auto * select_with_union_query_typed = query_ptr->as()) + { + auto & select_lists = select_with_union_query_typed->list_of_selects->as(); + + if (select_lists.children.size() == 1) + { + query_ptr = select_lists.children[0]; + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "UNION is not supported"); + } + } + else if (auto * select_query_typed = query_ptr->as()) + { + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Expected ASTSelectWithUnionQuery or ASTSelectQuery. Actual {}", + query_ptr_->formatForErrorMessage()); + } +} + +Block InterpreterSelectQueryAnalyzer::getSampleBlock() +{ + initializeQueryPlanIfNeeded(); + return query_plan.getCurrentDataStream().header; +} + +BlockIO InterpreterSelectQueryAnalyzer::execute() +{ + initializeQueryPlanIfNeeded(); + + QueryPlanOptimizationSettings optimization_settings; + BuildQueryPipelineSettings build_pipeline_settings; + auto pipeline_builder = query_plan.buildQueryPipeline(optimization_settings, build_pipeline_settings); + + BlockIO res; + res.pipeline = QueryPipelineBuilder::getPipeline(std::move(*pipeline_builder)); + + return res; +} + +void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() +{ + if (query_plan.isInitialized()) + return; + + auto query_tree_untyped = buildQueryTree(query_ptr, getContext()); + auto query_tree = std::static_pointer_cast(query_tree_untyped); + + auto * table_node = query_tree->getFrom()->as(); + if (!table_node) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only single table is supported"); + + query_tree_pass_manager.run(query_tree); + + ActionsDAGPtr action_dag = std::make_shared(); + ColumnsWithTypeAndName inputs; + + CollectSourceColumnsVisitor::Data data; + CollectSourceColumnsVisitor collect_source_columns_visitor(data); + collect_source_columns_visitor.visit(query_tree_untyped); + NameSet source_columns_set = std::move(data.source_columns_set); + + // std::cout << "DAG before " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl; + // std::cout << action_dag->dumpDAG() << std::endl; + + QueryTreeActionsVisitor visitor(action_dag, getContext()); + auto projection_action_dag_nodes = visitor.visit(query_tree->getProjectionNode()); + size_t projection_action_dag_nodes_size = projection_action_dag_nodes.size(); + + // std::cout << "Projection action dag nodes size " << projection_action_dag_nodes_size << std::endl; + // for (size_t i = 0; i < projection_action_dag_nodes_size; ++i) + // { + // std::cout << "DAG node " << projection_action_dag_nodes[i] << std::endl; + // } + + // std::cout << "DAG after " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl; + // std::cout << action_dag->dumpDAG() << std::endl; + + auto & projection_nodes = query_tree->getProjection().getNodes(); + size_t projection_nodes_size = projection_nodes.size(); + + if (projection_nodes_size != projection_action_dag_nodes_size) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "QueryTree projection nodes size mismatch. Expected {}. Actual {}", + projection_action_dag_nodes_size, + projection_nodes_size); + + NamesWithAliases projection_names; + for (size_t i = 0; i < projection_nodes_size; ++i) + { + auto & node = projection_nodes[i]; + const auto * action_dag_node = projection_action_dag_nodes[i]; + auto action_dag_node_name = action_dag_node->result_name; + + action_dag->getIndex().push_back(action_dag_node); + + if (node->hasAlias()) + projection_names.push_back({action_dag_node_name, node->getAlias()}); + else + projection_names.push_back({action_dag_node_name, action_dag_node_name}); + } + + action_dag->project(projection_names); + + // std::cout << "Final DAG " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl; + // std::cout << action_dag->dumpDAG() << std::endl; + // std::cout << "Names " << action_dag->dumpNames() << std::endl; + // std::cout << "Final DAG nodes " << std::endl; + // for (const auto & node : action_dag->getNodes()) + // { + // std::cout << "Node " << &node << " result name " << node.result_name << std::endl; + // } + + // std::cout << "Source columns " << source_columns_set.size() << std::endl; + // for (const auto & source_column : source_columns_set) + // std::cout << source_column << std::endl; + + auto current_context = getContext(); + size_t max_block_size = current_context->getSettingsRef().max_block_size; + size_t max_streams = current_context->getSettingsRef().max_threads; + + SelectQueryInfo query_info; + query_info.original_query = query_ptr; + query_info.query = query_ptr; + + auto from_stage = table_node->getStorage()->getQueryProcessingStage( + current_context, select_query_options.to_stage, table_node->getStorageSnapshot(), query_info); + + Names column_names(source_columns_set.begin(), source_columns_set.end()); + + if (column_names.empty() && table_node->getStorage()->getName() == "SystemOne") + column_names.push_back("dummy"); + + if (!column_names.empty()) + table_node->getStorage()->read( + query_plan, column_names, table_node->getStorageSnapshot(), query_info, getContext(), from_stage, max_block_size, max_streams); + + /// Create step which reads from empty source if storage has no data. + if (!query_plan.isInitialized()) + { + auto source_header = table_node->getStorageSnapshot()->getSampleBlockForColumns(column_names); + Pipe pipe(std::make_shared(source_header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource"); + query_plan.addStep(std::move(read_from_pipe)); + } + + auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), action_dag); + projection_step->setStepDescription("Projection"); + query_plan.addStep(std::move(projection_step)); +} + +} diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h new file mode 100644 index 00000000000..4f26cef801f --- /dev/null +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include + +#include +#include +#include + +namespace DB +{ + +class InterpreterSelectQueryAnalyzer : public IInterpreter, public WithContext +{ +public: + InterpreterSelectQueryAnalyzer( + const ASTPtr & query_ptr_, + const SelectQueryOptions & select_query_options_, + ContextPtr context); + + Block getSampleBlock(); + + BlockIO execute() override; + + bool supportsTransactions() const override { return true; } + +private: + void initializeQueryPlanIfNeeded(); + + ASTPtr query_ptr; + QueryPlan query_plan; + SelectQueryOptions select_query_options; + QueryTreePassManager query_tree_pass_manager; +}; + +} diff --git a/src/Parsers/ASTColumnsMatcher.cpp b/src/Parsers/ASTColumnsMatcher.cpp index 8f167f99b37..a887b239902 100644 --- a/src/Parsers/ASTColumnsMatcher.cpp +++ b/src/Parsers/ASTColumnsMatcher.cpp @@ -60,6 +60,11 @@ void ASTColumnsRegexpMatcher::setPattern(String pattern) DB::ErrorCodes::CANNOT_COMPILE_REGEXP); } +const std::shared_ptr & ASTColumnsRegexpMatcher::getMatcher() const +{ + return column_matcher; +} + bool ASTColumnsRegexpMatcher::isColumnMatching(const String & column_name) const { return RE2::PartialMatch(column_name, *column_matcher); @@ -114,4 +119,120 @@ void ASTColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatSt } } +ASTPtr ASTQualifiedColumnsRegexpMatcher::clone() const +{ + auto clone = std::make_shared(*this); + clone->cloneChildren(); + return clone; +} + +void ASTQualifiedColumnsRegexpMatcher::appendColumnName(WriteBuffer & ostr) const +{ + const auto & qualifier = children.at(0); + qualifier->appendColumnName(ostr); + writeCString(".COLUMNS(", ostr); + writeQuotedString(original_pattern, ostr); + writeChar(')', ostr); +} + +void ASTQualifiedColumnsRegexpMatcher::setPattern(String pattern) +{ + original_pattern = std::move(pattern); + column_matcher = std::make_shared(original_pattern, RE2::Quiet); + if (!column_matcher->ok()) + throw DB::Exception( + "COLUMNS pattern " + original_pattern + " cannot be compiled: " + column_matcher->error(), + DB::ErrorCodes::CANNOT_COMPILE_REGEXP); +} + +void ASTQualifiedColumnsRegexpMatcher::setMatcher(std::shared_ptr matcher) +{ + column_matcher = std::move(matcher); +} + +const std::shared_ptr & ASTQualifiedColumnsRegexpMatcher::getMatcher() const +{ + return column_matcher; +} + +void ASTQualifiedColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state) const +{ + hash_state.update(original_pattern.size()); + hash_state.update(original_pattern); + IAST::updateTreeHashImpl(hash_state); +} + +void ASTQualifiedColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + const auto & qualifier = children.at(0); + qualifier->formatImpl(settings, state, frame); + + settings.ostr << (settings.hilite ? hilite_keyword : ""); + settings.ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "("; + settings.ostr << quoteString(original_pattern); + settings.ostr << ")"; + + /// Format column transformers + for (const auto & child : children) + { + settings.ostr << ' '; + child->formatImpl(settings, state, frame); + } +} + +ASTPtr ASTQualifiedColumnsListMatcher::clone() const +{ + auto clone = std::make_shared(*this); + clone->column_list = column_list->clone(); + clone->cloneChildren(); + return clone; +} + +void ASTQualifiedColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const +{ + const auto & qualifier = children.at(0); + qualifier->appendColumnName(ostr); + writeCString(".COLUMNS(", ostr); + + for (auto it = column_list->children.begin(); it != column_list->children.end(); ++it) + { + if (it != column_list->children.begin()) + writeCString(", ", ostr); + + (*it)->appendColumnName(ostr); + } + writeChar(')', ostr); +} + +void ASTQualifiedColumnsListMatcher::updateTreeHashImpl(SipHash & hash_state) const +{ + column_list->updateTreeHash(hash_state); + IAST::updateTreeHashImpl(hash_state); +} + +void ASTQualifiedColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + const auto & qualifier = children.at(0); + qualifier->formatImpl(settings, state, frame); + + settings.ostr << (settings.hilite ? hilite_keyword : "") << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "("; + + for (ASTs::const_iterator it = column_list->children.begin(); it != column_list->children.end(); ++it) + { + if (it != column_list->children.begin()) + { + settings.ostr << ", "; + } + (*it)->formatImpl(settings, state, frame); + } + settings.ostr << ")"; + + /// Format column transformers + for (const auto & child : children) + { + settings.ostr << ' '; + child->formatImpl(settings, state, frame); + } +} + } diff --git a/src/Parsers/ASTColumnsMatcher.h b/src/Parsers/ASTColumnsMatcher.h index 5aaf3cbe30d..7ce246608b9 100644 --- a/src/Parsers/ASTColumnsMatcher.h +++ b/src/Parsers/ASTColumnsMatcher.h @@ -24,6 +24,7 @@ public: void appendColumnName(WriteBuffer & ostr) const override; void setPattern(String pattern); + const std::shared_ptr & getMatcher() const; bool isColumnMatching(const String & column_name) const; void updateTreeHashImpl(SipHash & hash_state) const override; @@ -49,5 +50,39 @@ protected: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; }; +/// Same as ASTColumnsRegexpMatcher. Qualified identifier is first child. +class ASTQualifiedColumnsRegexpMatcher : public IAST +{ +public: + String getID(char) const override { return "QualifiedColumnsRegexpMatcher"; } + ASTPtr clone() const override; + + void appendColumnName(WriteBuffer & ostr) const override; + const std::shared_ptr & getMatcher() const; + void setPattern(String pattern); + void setMatcher(std::shared_ptr matcher); + void updateTreeHashImpl(SipHash & hash_state) const override; + +protected: + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; + +private: + std::shared_ptr column_matcher; + String original_pattern; +}; + +/// Same as ASTColumnsListMatcher. Qualified identifier is first child. +class ASTQualifiedColumnsListMatcher : public IAST +{ +public: + String getID(char) const override { return "QualifiedColumnsListMatcher"; } + ASTPtr clone() const override; + void appendColumnName(WriteBuffer & ostr) const override; + void updateTreeHashImpl(SipHash & hash_state) const override; + + ASTPtr column_list; +protected: + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; } diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 71207724a89..118c22b463f 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -270,6 +270,11 @@ void ASTColumnsExceptTransformer::setPattern(String pattern) DB::ErrorCodes::CANNOT_COMPILE_REGEXP); } +const std::shared_ptr & ASTColumnsExceptTransformer::getMatcher() const +{ + return column_matcher; +} + bool ASTColumnsExceptTransformer::isColumnMatching(const String & column_name) const { return RE2::PartialMatch(column_name, *column_matcher); diff --git a/src/Parsers/ASTColumnsTransformers.h b/src/Parsers/ASTColumnsTransformers.h index 0f16f6b93e7..5179726e8cb 100644 --- a/src/Parsers/ASTColumnsTransformers.h +++ b/src/Parsers/ASTColumnsTransformers.h @@ -60,6 +60,7 @@ public: } void transform(ASTs & nodes) const override; void setPattern(String pattern); + const std::shared_ptr & getMatcher() const; bool isColumnMatching(const String & column_name) const; void appendColumnName(WriteBuffer & ostr) const override; void updateTreeHashImpl(SipHash & hash_state) const override; diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index ea9ccf5a4f4..f802751b0ec 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -15,6 +15,7 @@ public: { ParsedAST, /// 'EXPLAIN AST SELECT ...' AnalyzedSyntax, /// 'EXPLAIN SYNTAX SELECT ...' + QueryTree, /// 'EXPLAIN QUERYTREE SELECT ...' QueryPlan, /// 'EXPLAIN SELECT ...' QueryPipeline, /// 'EXPLAIN PIPELINE ...' QueryEstimates, /// 'EXPLAIN ESTIMATE ...' @@ -109,6 +110,7 @@ private: { case ParsedAST: return "EXPLAIN AST"; case AnalyzedSyntax: return "EXPLAIN SYNTAX"; + case QueryTree: return "EXPLAIN QUERYTREE"; case QueryPlan: return "EXPLAIN"; case QueryPipeline: return "EXPLAIN PIPELINE"; case QueryEstimates: return "EXPLAIN ESTIMATE"; diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 6d5089f802e..5756fb9ba86 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -24,6 +24,8 @@ public: bool compute_after_window_functions = false; + bool is_lambda_function = false; + // We have to make these fields ASTPtr because this is what the visitors // expect. Some of them take const ASTPtr & (makes no sense), and some // take ASTPtr & and modify it. I don't understand how the latter is diff --git a/src/Parsers/ASTIdentifier.h b/src/Parsers/ASTIdentifier.h index 14e2fcef39d..c9712d578e0 100644 --- a/src/Parsers/ASTIdentifier.h +++ b/src/Parsers/ASTIdentifier.h @@ -49,9 +49,10 @@ public: void restoreTable(); // TODO(ilezhankin): get rid of this std::shared_ptr createTable() const; // returns |nullptr| if identifier is not table. -protected: String full_name; std::vector name_parts; + +protected: std::shared_ptr semantic; /// pimpl void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 88784329ece..c4e07ea2e15 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1192,54 +1192,6 @@ bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return true; } - -bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserKeyword columns("COLUMNS"); - ParserList columns_p(std::make_unique(false, true), std::make_unique(TokenType::Comma), false); - ParserStringLiteral regex; - - if (!columns.ignore(pos, expected)) - return false; - - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - - ASTPtr column_list; - ASTPtr regex_node; - if (!columns_p.parse(pos, column_list, expected) && !regex.parse(pos, regex_node, expected)) - return false; - - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; - - ASTPtr res; - if (column_list) - { - auto list_matcher = std::make_shared(); - list_matcher->column_list = column_list; - res = list_matcher; - } - else - { - auto regexp_matcher = std::make_shared(); - regexp_matcher->setPattern(regex_node->as().value.get()); - res = regexp_matcher; - } - - ParserColumnsTransformers transformers_p(allowed_transformers); - ASTPtr transformer; - while (transformers_p.parse(pos, transformer, expected)) - { - res->children.push_back(transformer); - } - node = std::move(res); - return true; -} - - bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword apply("APPLY"); @@ -1488,6 +1440,122 @@ bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return true; } +/// Parse (columns_list) or ('REGEXP'). +static bool parseColumnsMatcherBody(IParser::Pos & pos, ASTPtr & node, Expected & expected, ParserColumnsTransformers::ColumnTransformers allowed_transformers) +{ + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + ParserList columns_p(std::make_unique(false, true), std::make_unique(TokenType::Comma), false); + ParserStringLiteral regex; + + ASTPtr column_list; + ASTPtr regex_node; + if (!columns_p.parse(pos, column_list, expected) && !regex.parse(pos, regex_node, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + ASTPtr res; + if (column_list) + { + auto list_matcher = std::make_shared(); + list_matcher->column_list = column_list; + res = list_matcher; + } + else + { + auto regexp_matcher = std::make_shared(); + regexp_matcher->setPattern(regex_node->as().value.get()); + res = regexp_matcher; + } + + ParserColumnsTransformers transformers_p(allowed_transformers); + ASTPtr transformer; + while (transformers_p.parse(pos, transformer, expected)) + { + res->children.push_back(transformer); + } + + node = std::move(res); + return true; +} + +bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword columns("COLUMNS"); + + if (!columns.ignore(pos, expected)) + return false; + + return parseColumnsMatcherBody(pos, node, expected, allowed_transformers); +} + +bool ParserQualifiedColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserCompoundIdentifier(true, true).parse(pos, node, expected)) + return false; + + auto identifier_node = node; + const auto & identifier_node_typed = identifier_node->as(); + + /// ParserCompoundIdentifier parse identifier.COLUMNS + if (identifier_node_typed.name_parts.size() == 1 || identifier_node_typed.name_parts.back() != "COLUMNS") + return false; + + /// TODO: ASTTableIdentifier can contain only 2 parts + + if (identifier_node_typed.name_parts.size() == 2) + { + auto table_name = identifier_node_typed.name_parts[0]; + identifier_node = std::make_shared(table_name); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected identifier to contain no more than 2 parts. Actual {}", + identifier_node_typed.full_name); + } + + if (!parseColumnsMatcherBody(pos, node, expected, allowed_transformers)) + return false; + + if (auto * columns_list_matcher = node->as()) + { + auto result = std::make_shared(); + result->column_list = std::move(columns_list_matcher->column_list); + + result->children.reserve(columns_list_matcher->children.size() + 1); + result->children.push_back(std::move(identifier_node)); + + for (auto && child : columns_list_matcher->children) + result->children.push_back(std::move(child)); + + node = result; + } + else if (auto * column_regexp_matcher = node->as()) + { + auto result = std::make_shared(); + result->setMatcher(column_regexp_matcher->getMatcher()); + + result->children.reserve(column_regexp_matcher->children.size() + 1); + result->children.push_back(std::move(identifier_node)); + + for (auto && child : column_regexp_matcher->children) + result->children.push_back(std::move(child)); + + node = result; + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Qualified COLUMNS matcher expected to be list or regexp"); + } + + return true; +} bool ParserSubstitution::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index f538555f0c1..8a9647dc86f 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -104,7 +104,7 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -/** COLUMNS('') +/** COLUMNS(columns_names) or COLUMNS('') */ class ParserColumnsMatcher : public IParserBase { @@ -121,6 +121,23 @@ protected: ColumnTransformers allowed_transformers; }; +/** Qualified columns matcher identifier.COLUMNS(columns_names) or identifier.COLUMNS('') + */ +class ParserQualifiedColumnsMatcher : public IParserBase +{ +public: + using ColumnTransformers = ParserColumnsTransformers::ColumnTransformers; + explicit ParserQualifiedColumnsMatcher(ColumnTransformers allowed_transformers_ = ParserColumnsTransformers::AllTransformers) + : allowed_transformers(allowed_transformers_) + {} + +protected: + const char * getName() const override { return "qualified COLUMNS matcher"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + ColumnTransformers allowed_transformers; +}; + // Allows to make queries like SELECT SUM() FILTER(WHERE ) FROM ... class ParserFilterClause : public IParserBase { diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index d32d4444c36..52d82a29118 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -19,6 +19,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_ast("AST"); ParserKeyword s_explain("EXPLAIN"); ParserKeyword s_syntax("SYNTAX"); + ParserKeyword s_query_tree("QUERYTREE"); ParserKeyword s_pipeline("PIPELINE"); ParserKeyword s_plan("PLAN"); ParserKeyword s_estimates("ESTIMATE"); @@ -33,6 +34,8 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected kind = ASTExplainQuery::ExplainKind::ParsedAST; else if (s_syntax.ignore(pos, expected)) kind = ASTExplainQuery::ExplainKind::AnalyzedSyntax; + else if (s_query_tree.ignore(pos, expected)) + kind = ASTExplainQuery::ExplainKind::QueryTree; else if (s_pipeline.ignore(pos, expected)) kind = ASTExplainQuery::ExplainKind::QueryPipeline; else if (s_plan.ignore(pos, expected)) @@ -84,6 +87,15 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected explain_query->setTableFunction(table_function); explain_query->setTableOverride(table_override); } + if (kind == ASTExplainQuery::ExplainKind::QueryTree) + { + if (select_p.parse(pos, query, expected)) + explain_query->setExplainedQuery(std::move(query)); + else + { + return false; + } + } else if (kind == ASTExplainQuery::ExplainKind::CurrentTransaction) { /// Nothing to parse @@ -103,7 +115,9 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected explain_query->setExplainedQuery(std::move(query)); } else + { return false; + } node = std::move(explain_query); return true; diff --git a/src/Parsers/SelectUnionMode.cpp b/src/Parsers/SelectUnionMode.cpp new file mode 100644 index 00000000000..4a7fac0fca1 --- /dev/null +++ b/src/Parsers/SelectUnionMode.cpp @@ -0,0 +1,24 @@ +#include + + +namespace DB +{ + +const char * toString(SelectUnionMode mode) +{ + switch (mode) + { + case SelectUnionMode::ALL: + return "ALL"; + case SelectUnionMode::DISTINCT: + return "DISTINCT"; + case SelectUnionMode::EXCEPT: + return "EXCEPT"; + case SelectUnionMode::INTERSECT: + return "INTERSECT"; + case SelectUnionMode::Unspecified: + return "Unspecified"; + } +} + +} diff --git a/src/Parsers/SelectUnionMode.h b/src/Parsers/SelectUnionMode.h index ca3637612aa..5c72ce65eb2 100644 --- a/src/Parsers/SelectUnionMode.h +++ b/src/Parsers/SelectUnionMode.h @@ -18,6 +18,8 @@ enum class SelectUnionMode INTERSECT_DISTINCT }; +const char * toString(SelectUnionMode mode); + using SelectUnionModes = std::vector; using SelectUnionModesSet = std::unordered_set; From bd11c617b88b3ddd7c809031f839ae61e4bce4a2 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Jul 2022 13:31:58 +0200 Subject: [PATCH 051/188] Added tests --- .../02337_analyzer_columns_basic.reference | 46 +++++ .../02337_analyzer_columns_basic.sql | 98 +++++++++ .../02338_analyzer_constants_basic.reference | 35 ++++ .../02338_analyzer_constants_basic.sql | 42 ++++ .../02339_analyzer_matcher_basic.reference | 94 +++++++++ .../02339_analyzer_matcher_basic.sql | 186 +++++++++++++++++ .../02340_analyzer_functions.reference | 11 + .../0_stateless/02340_analyzer_functions.sql | 28 +++ .../02341_analyzer_aliases_basics.reference | 19 ++ .../02341_analyzer_aliases_basics.sql | 50 +++++ .../02342_analyzer_compound_types.reference | 142 +++++++++++++ .../02342_analyzer_compound_types.sql | 195 ++++++++++++++++++ 12 files changed, 946 insertions(+) create mode 100644 tests/queries/0_stateless/02337_analyzer_columns_basic.reference create mode 100644 tests/queries/0_stateless/02337_analyzer_columns_basic.sql create mode 100644 tests/queries/0_stateless/02338_analyzer_constants_basic.reference create mode 100644 tests/queries/0_stateless/02338_analyzer_constants_basic.sql create mode 100644 tests/queries/0_stateless/02339_analyzer_matcher_basic.reference create mode 100644 tests/queries/0_stateless/02339_analyzer_matcher_basic.sql create mode 100644 tests/queries/0_stateless/02340_analyzer_functions.reference create mode 100644 tests/queries/0_stateless/02340_analyzer_functions.sql create mode 100644 tests/queries/0_stateless/02341_analyzer_aliases_basics.reference create mode 100644 tests/queries/0_stateless/02341_analyzer_aliases_basics.sql create mode 100644 tests/queries/0_stateless/02342_analyzer_compound_types.reference create mode 100644 tests/queries/0_stateless/02342_analyzer_compound_types.sql diff --git a/tests/queries/0_stateless/02337_analyzer_columns_basic.reference b/tests/queries/0_stateless/02337_analyzer_columns_basic.reference new file mode 100644 index 00000000000..1482c79b602 --- /dev/null +++ b/tests/queries/0_stateless/02337_analyzer_columns_basic.reference @@ -0,0 +1,46 @@ +Empty from section +dummy UInt8 +0 +-- +dummy UInt8 +0 +-- +dummy UInt8 +0 +Table access without table name qualification +id UInt64 +0 +-- +value String +Value +-- +id UInt64 +value String +0 Value +Table access with table name qualification +id UInt64 +0 +-- +value String +Value +-- +id UInt64 +value String +0 Value +-- +id UInt64 +value String +0 Value +Table access with database and table name qualification +-- +id UInt64 +value String +0 Value +-- +id UInt64 +value String +0 Value +-- +id UInt64 +value String +0 Value diff --git a/tests/queries/0_stateless/02337_analyzer_columns_basic.sql b/tests/queries/0_stateless/02337_analyzer_columns_basic.sql new file mode 100644 index 00000000000..0f9efdb95cd --- /dev/null +++ b/tests/queries/0_stateless/02337_analyzer_columns_basic.sql @@ -0,0 +1,98 @@ +SET use_analyzer = 1; + +-- Empty from section + +SELECT 'Empty from section'; + +DESCRIBE (SELECT dummy); +SELECT dummy; + +SELECT '--'; + +DESCRIBE (SELECT one.dummy); +SELECT one.dummy; + +SELECT '--'; + +DESCRIBE (SELECT system.one.dummy); +SELECT system.one.dummy; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +SELECT 'Table access without table name qualification'; + +SELECT test_id FROM test_table; -- { serverError 47 } + +DESCRIBE (SELECT id FROM test_table); +SELECT id FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value FROM test_table); +SELECT value FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT id, value FROM test_table); +SELECT id, value FROM test_table; + +SELECT 'Table access with table name qualification'; + +DESCRIBE (SELECT test_table.id FROM test_table); +SELECT test_table.id FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT test_table.value FROM test_table); +SELECT test_table.value FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT test_table.id, test_table.value FROM test_table); +SELECT test_table.id, test_table.value FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT test.id, test.value FROM test_table AS test); +SELECT test.id, test.value FROM test_table AS test; + +DROP TABLE test_table; + +SELECT 'Table access with database and table name qualification'; + +DROP DATABASE IF EXISTS 02337_db; +CREATE DATABASE 02337_db; + +DROP TABLE IF EXISTS 02337_db.test_table; +CREATE TABLE 02337_db.test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO 02337_db.test_table VALUES (0, 'Value'); + +SELECT '--'; + +DESCRIBE (SELECT test_table.id, test_table.value FROM 02337_db.test_table); +SELECT test_table.id, test_table.value FROM 02337_db.test_table; + +SELECT '--'; + +DESCRIBE (SELECT 02337_db.test_table.id, 02337_db.test_table.value FROM 02337_db.test_table); +SELECT 02337_db.test_table.id, 02337_db.test_table.value FROM 02337_db.test_table; + +SELECT '--'; + +DESCRIBE (SELECT test_table.id, test_table.value FROM 02337_db.test_table AS test_table); +SELECT test_table.id, test_table.value FROM 02337_db.test_table AS test_table; + +DROP TABLE 02337_db.test_table; +DROP DATABASE 02337_db; diff --git a/tests/queries/0_stateless/02338_analyzer_constants_basic.reference b/tests/queries/0_stateless/02338_analyzer_constants_basic.reference new file mode 100644 index 00000000000..32f8a5eb124 --- /dev/null +++ b/tests/queries/0_stateless/02338_analyzer_constants_basic.reference @@ -0,0 +1,35 @@ +1 UInt8 +1 +-- +\'test\' String +test +-- +1 UInt8 +\'test\' String +1 test +-- +1 UInt8 +\'test\' String +[1, 2, 3] Array(UInt8) +1 test [1,2,3] +-- +1 UInt8 +\'test\' String +[1, 2, 3] Array(UInt8) +[\'1\', \'2\', \'3\'] Array(String) +1 test [1,2,3] ['1','2','3'] +-- +NULL Nullable(Nothing) +\N +-- +(1, 1) Tuple(UInt8, UInt8) +(1,1) +-- +[(1, 1)] Array(Tuple(UInt8, UInt8)) +[(1,1)] +NULL Nullable(Nothing) +1 UInt8 +\'test\' String +[1, 2, 3] Array(UInt8) +[(1, 1), (1, 1)] Array(Tuple(UInt8, UInt8)) +\N 1 test [1,2,3] [(1,1),(1,1)] diff --git a/tests/queries/0_stateless/02338_analyzer_constants_basic.sql b/tests/queries/0_stateless/02338_analyzer_constants_basic.sql new file mode 100644 index 00000000000..56f82763f1b --- /dev/null +++ b/tests/queries/0_stateless/02338_analyzer_constants_basic.sql @@ -0,0 +1,42 @@ +SET use_analyzer = 1; + +DESCRIBE (SELECT 1); +SELECT 1; + +SELECT '--'; + +DESCRIBE (SELECT 'test'); +SELECT 'test'; + +SELECT '--'; + +DESCRIBE (SELECT 1, 'test'); +SELECT 1, 'test'; + +SELECT '--'; + +DESCRIBE (SELECT 1, 'test', [1, 2, 3]); +SELECT 1, 'test', [1, 2, 3]; + +SELECT '--'; + +DESCRIBE (SELECT 1, 'test', [1, 2, 3], ['1', '2', '3']); +SELECT 1, 'test', [1, 2, 3], ['1', '2', '3']; + +SELECT '--'; + +DESCRIBE (SELECT NULL); +SELECT NULL; + +SELECT '--'; + +DESCRIBE (SELECT (1, 1)); +SELECT (1, 1); + +SELECT '--'; + +DESCRIBE (SELECT [(1, 1)]); +SELECT [(1, 1)]; + +DESCRIBE (SELECT NULL, 1, 'test', [1, 2, 3], [(1, 1), (1, 1)]); +SELECT NULL, 1, 'test', [1, 2, 3], [(1, 1), (1, 1)]; diff --git a/tests/queries/0_stateless/02339_analyzer_matcher_basic.reference b/tests/queries/0_stateless/02339_analyzer_matcher_basic.reference new file mode 100644 index 00000000000..001fe62d4f9 --- /dev/null +++ b/tests/queries/0_stateless/02339_analyzer_matcher_basic.reference @@ -0,0 +1,94 @@ +Matchers without FROM section +dummy UInt8 +0 +-- +dummy UInt8 +0 +-- +dummy UInt8 +0 +Unqualified matchers +id UInt64 +value String +0 Value +-- +id UInt64 +0 +-- +id UInt64 +value String +0 Value +-- +id UInt64 +value String +0 Value +Table qualified matchers +id UInt64 +value String +0 Value +-- +id UInt64 +0 +-- +id UInt64 +value String +0 Value +-- +id UInt64 +value String +0 Value +Database and table qualified matchers +APPLY transformer +-- +toString(id) String +toString(value) String +0 Value +-- +toString(id) String +toString(value) String +0 Value +-- +length(toString(id)) UInt64 +length(toString(value)) UInt64 +1 5 +-- +length(toString(id)) UInt64 +length(toString(value)) UInt64 +1 5 +-- +id UInt64 +value String +0 Value +EXCEPT transformer +-- +value String +Value +-- +value String +Value +-- +toString(value) String +Value +-- +toString(value) String +Value +REPLACE transformer +-- +5 UInt8 +value String +5 Value +-- +5 UInt8 +value String +5 Value +-- +5 UInt8 +6 UInt8 +-- +5 UInt8 +6 UInt8 +Combine EXCEPT, REPLACE, APPLY transformers +-- +\'6\' String +-- +\'6\' String diff --git a/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql b/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql new file mode 100644 index 00000000000..5d479293084 --- /dev/null +++ b/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql @@ -0,0 +1,186 @@ +SET use_analyzer = 1; + +SELECT 'Matchers without FROM section'; + +DESCRIBE (SELECT *); +SELECT *; + +SELECT '--'; + +DESCRIBE (SELECT COLUMNS(dummy)); +SELECT COLUMNS(dummy); + +SELECT '--'; + +DESCRIBE (SELECT COLUMNS('d')); +SELECT COLUMNS('d'); + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +SELECT 'Unqualified matchers'; + +DESCRIBE (SELECT * FROM test_table); +SELECT * FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT COLUMNS(id) FROM test_table); +SELECT COLUMNS(id) FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT COLUMNS(id), COLUMNS(value) FROM test_table); +SELECT COLUMNS(id), COLUMNS(value) FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT COLUMNS('i'), COLUMNS('v') FROM test_table); +SELECT COLUMNS('i'), COLUMNS('v') FROM test_table; + +SELECT 'Table qualified matchers'; + +DESCRIBE (SELECT test_table.* FROM test_table); +SELECT test_table.* FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT test_table.COLUMNS(id) FROM test_table); +SELECT test_table.COLUMNS(id) FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT test_table.COLUMNS(id), test_table.COLUMNS(value) FROM test_table); +SELECT test_table.COLUMNS(id), test_table.COLUMNS(value) FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT test_table.COLUMNS('i'), test_table.COLUMNS('v') FROM test_table); +SELECT test_table.COLUMNS('i'), test_table.COLUMNS('v') FROM test_table; + +SELECT 'Database and table qualified matchers'; + +DROP DATABASE IF EXISTS 02339_db; +CREATE DATABASE 02339_db; + +DROP TABLE IF EXISTS 02339_db.test_table; +CREATE TABLE 02339_db.test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO 02339_db.test_table VALUES (0, 'Value'); + +-- TODO: Qualified COLUMNS where identifier has more than 2 parts are not supported on parser level + +-- SELECT '--'; + +-- DESCRIBE (SELECT 02339_db.test_table.* FROM 02339_db.test_table); +-- SELECT 02339_db.test_table.* FROM 02339_db.test_table; + +-- SELECT '--'; + +-- DESCRIBE (SELECT 02339_db.test_table.COLUMNS(id) FROM 02339_db.test_table); +-- SELECT 02339_db.test_table.COLUMNS(id) FROM 02339_db.test_table; + +-- SELECT '--'; + +-- DESCRIBE (SELECT 02339_db.test_table.COLUMNS(id), 02339_db.test_table.COLUMNS(value) FROM 02339_db.test_table); +-- SELECT 02339_db.test_table.COLUMNS(id), 02339_db.test_table.COLUMNS(value) FROM 02339_db.test_table; + +-- SELECT '--'; + +-- DESCRIBE (SELECT 02339_db.test_table.COLUMNS('i'), 02339_db.test_table.COLUMNS('v') FROM 02339_db.test_table); +-- SELECT 02339_db.test_table.COLUMNS('i'), 02339_db.test_table.COLUMNS('v') FROM 02339_db.test_table; + +DROP TABLE 02339_db.test_table; +DROP DATABASE 02339_db; + +SELECT 'APPLY transformer'; + +SELECT '--'; + +DESCRIBE (SELECT * APPLY toString FROM test_table); +SELECT * APPLY toString FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT * APPLY (x -> toString(x)) FROM test_table); +SELECT * APPLY (x -> toString(x)) FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT * APPLY (x -> toString(x)) APPLY (x -> length(x)) FROM test_table); +SELECT * APPLY (x -> toString(x)) APPLY (x -> length(x)) FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT * APPLY (x -> toString(x)) APPLY length FROM test_table); +SELECT * APPLY (x -> toString(x)) APPLY length FROM test_table; + +SELECT '--'; +DESCRIBE (SELECT * FROM test_table); +SELECT * FROM test_table; + +SELECT 'EXCEPT transformer'; + +SELECT '--'; + +DESCRIBE (SELECT * EXCEPT (id) FROM test_table); +SELECT * EXCEPT (id) FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT COLUMNS(id, value) EXCEPT (id) FROM test_table); +SELECT COLUMNS(id, value) EXCEPT (id) FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT * EXCEPT (id) APPLY toString FROM test_table); +SELECT * EXCEPT (id) APPLY toString FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT COLUMNS(id, value) EXCEPT (id) APPLY toString FROM test_table); +SELECT COLUMNS(id, value) EXCEPT (id) APPLY toString FROM test_table; + +SELECT 'REPLACE transformer'; + +SELECT '--'; + +DESCRIBE (SELECT * REPLACE (5 AS id) FROM test_table); +SELECT * REPLACE (5 AS id) FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT COLUMNS(id, value) REPLACE (5 AS id) FROM test_table); +SELECT COLUMNS(id, value) REPLACE (5 AS id) FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT * REPLACE (5 AS id, 6 as value) FROM test_table); +SELECT * REPLACE (5 AS id, 6 as value) FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT COLUMNS(id, value) REPLACE (5 AS id, 6 as value) FROM test_table); +SELECT COLUMNS(id, value) REPLACE (5 AS id, 6 as value) FROM test_table; + +SELECT 'Combine EXCEPT, REPLACE, APPLY transformers'; + +SELECT '--'; + +DESCRIBE (SELECT * EXCEPT id REPLACE (5 AS id, 6 as value) APPLY toString FROM test_table); +SELECT * EXCEPT id REPLACE (5 AS id, 6 as value) APPLY toString FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT COLUMNS(id, value) EXCEPT id REPLACE (5 AS id, 6 as value) APPLY toString FROM test_table); +SELECT COLUMNS(id, value) EXCEPT id REPLACE (5 AS id, 6 as value) APPLY toString FROM test_table; diff --git a/tests/queries/0_stateless/02340_analyzer_functions.reference b/tests/queries/0_stateless/02340_analyzer_functions.reference new file mode 100644 index 00000000000..29dcb813557 --- /dev/null +++ b/tests/queries/0_stateless/02340_analyzer_functions.reference @@ -0,0 +1,11 @@ +2 UInt16 +2 +-- +plus(dummy, dummy) UInt16 +0 +-- +plus(id, length(value)) UInt64 +5 +-- +concat(concat(toString(id), \'_\'), value) String +0_Value diff --git a/tests/queries/0_stateless/02340_analyzer_functions.sql b/tests/queries/0_stateless/02340_analyzer_functions.sql new file mode 100644 index 00000000000..b1bfeabd836 --- /dev/null +++ b/tests/queries/0_stateless/02340_analyzer_functions.sql @@ -0,0 +1,28 @@ +SET use_analyzer = 1; + +DESCRIBE (SELECT 1 + 1); +SELECT 1 + 1; + +SELECT '--'; + +DESCRIBE (SELECT dummy + dummy); +SELECT dummy + dummy; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +SELECT '--'; + +DESCRIBE (SELECT id + length(value) FROM test_table); +SELECT id + length(value) FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT concat(concat(toString(id), '_'), (value)) FROM test_table); +SELECT concat(concat(toString(id), '_'), (value)) FROM test_table; diff --git a/tests/queries/0_stateless/02341_analyzer_aliases_basics.reference b/tests/queries/0_stateless/02341_analyzer_aliases_basics.reference new file mode 100644 index 00000000000..ea60915f479 --- /dev/null +++ b/tests/queries/0_stateless/02341_analyzer_aliases_basics.reference @@ -0,0 +1,19 @@ +Aliases to constants +1 1 +4 2 1 3 4 +1 +1 1 +1 1 2 +1 2 1 +3 6 +Aliases to columns +0 0 0 +0 Value 0 Value +0 Value +Alias conflict with identifier inside expression +0 +1 +3 +Alias setting prefer_column_name_to_alias +0 0 +0 Value diff --git a/tests/queries/0_stateless/02341_analyzer_aliases_basics.sql b/tests/queries/0_stateless/02341_analyzer_aliases_basics.sql new file mode 100644 index 00000000000..4abd8bc55f7 --- /dev/null +++ b/tests/queries/0_stateless/02341_analyzer_aliases_basics.sql @@ -0,0 +1,50 @@ +SET use_analyzer = 1; + +SELECT 'Aliases to constants'; + +SELECT 1 as a, a; +SELECT (c + 1) as d, (a + 1) as b, 1 AS a, (b + 1) as c, d; + +WITH 1 as a SELECT a; +WITH a as b SELECT 1 as a, b; + +SELECT 1 AS x, x, x + 1; +SELECT x, x + 1, 1 AS x; +SELECT x, 1 + (2 + (3 AS x)); + +SELECT a AS b, b AS a; -- { serverError 174 } + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +SELECT 'Aliases to columns'; + +SELECT id_alias_2, id AS id_alias, id_alias as id_alias_2 FROM test_table; +SELECT id_1, value_1, id as id_1, value as value_1 FROM test_table; + +WITH value_1 as value_2, id_1 as id_2, id AS id_1, value AS value_1 SELECT id_2, value_2 FROM test_table; + +SELECT (id + b) AS id, id as b FROM test_table; -- { serverError 174 } +SELECT (1 + b + 1 + id) AS id, b as c, id as b FROM test_table; -- { serverError 174 } + +SELECT 'Alias conflict with identifier inside expression'; + +SELECT id AS id FROM test_table; +SELECT (id + 1) AS id FROM test_table; +SELECT (id + 1 + 1 + 1 + id) AS id FROM test_table; + +SELECT 'Alias setting prefer_column_name_to_alias'; + +SELECT id AS value, value FROM test_table; + +SET prefer_column_name_to_alias = 1; +SELECT id AS value, value FROM test_table; +SET prefer_column_name_to_alias = 0; + +DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02342_analyzer_compound_types.reference b/tests/queries/0_stateless/02342_analyzer_compound_types.reference new file mode 100644 index 00000000000..6d6cfb5e579 --- /dev/null +++ b/tests/queries/0_stateless/02342_analyzer_compound_types.reference @@ -0,0 +1,142 @@ +Constant tuple +(1,'Value') 1 Value +(1,'Value') 1 Value +(1,'Value') 1 +(1,'Value') Value +(1,'Value') 1 +(1,'Value') Value +Tuple +-- +id UInt64 +value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +0 (('value_0_level_1','value_1_level_1'),'value_1_level_0') +-- +id UInt64 +value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +0 (('value_0_level_1','value_1_level_1'),'value_1_level_0') +-- +value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +value.value_1_level_0 String +('value_0_level_1','value_1_level_1') value_1_level_0 +-- +value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +tupleElement(value, \'value_0_level_0\') Tuple(value_0_level_1 String, value_1_level_1 String) +tupleElement(value, \'value_1_level_0\') String +(('value_0_level_1','value_1_level_1'),'value_1_level_0') ('value_0_level_1','value_1_level_1') value_1_level_0 +-- +value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +tupleElement(value, \'value_0_level_0\') Tuple(value_0_level_1 String, value_1_level_1 String) +tupleElement(value, \'value_1_level_0\') String +(('value_0_level_1','value_1_level_1'),'value_1_level_0') ('value_0_level_1','value_1_level_1') value_1_level_0 +-- +value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +toString(tupleElement(value, \'value_0_level_0\')) String +toString(tupleElement(value, \'value_1_level_0\')) String +(('value_0_level_1','value_1_level_1'),'value_1_level_0') (\'value_0_level_1\',\'value_1_level_1\') value_1_level_0 +-- +tupleElement(value, \'value_0_level_0\') Tuple(value_0_level_1 String, value_1_level_1 String) +tupleElement(value, \'value_1_level_0\') String +('value_0_level_1','value_1_level_1') value_1_level_0 +-- +toString(tupleElement(value, \'value_0_level_0\')) String +toString(tupleElement(value, \'value_1_level_0\')) String +(\'value_0_level_1\',\'value_1_level_1\') value_1_level_0 +-- +value.value_0_level_0.value_0_level_1 String +value.value_0_level_0.value_1_level_1 String +value_0_level_1 value_1_level_1 +-- +value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +tupleElement(value.value_0_level_0, \'value_0_level_1\') String +tupleElement(value.value_0_level_0, \'value_1_level_1\') String +('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 +-- +value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +tupleElement(value.value_0_level_0, \'value_0_level_1\') String +tupleElement(value.value_0_level_0, \'value_1_level_1\') String +('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 +-- +value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String +toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String +('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 +-- +tupleElement(value.value_0_level_0, \'value_0_level_1\') String +tupleElement(value.value_0_level_0, \'value_1_level_1\') String +value_0_level_1 value_1_level_1 +-- +toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String +toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String +value_0_level_1 value_1_level_1 +Array of tuples +id UInt64 +value.value_0_level_0 Array(Tuple(value_0_level_1 String, value_1_level_1 String)) +value.value_1_level_0 Array(String) +0 [('value_0_level_1','value_1_level_1')] ['value_1_level_0'] +-- +value.value_0_level_0 Array(Tuple(value_0_level_1 String, value_1_level_1 String)) +value.value_1_level_0 Array(String) +[('value_0_level_1','value_1_level_1')] ['value_1_level_0'] +-- +value.value_0_level_0.value_0_level_1 Array(String) +value.value_0_level_0.value_1_level_1 Array(String) +['value_0_level_1'] ['value_1_level_1'] +-- +value.value_0_level_0 Array(Tuple(value_0_level_1 String, value_1_level_1 String)) +tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(String) +tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(String) +[('value_0_level_1','value_1_level_1')] ['value_0_level_1'] ['value_1_level_1'] +-- +value.value_0_level_0 Array(Tuple(value_0_level_1 String, value_1_level_1 String)) +tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(String) +tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(String) +[('value_0_level_1','value_1_level_1')] ['value_0_level_1'] ['value_1_level_1'] +-- +value.value_0_level_0 Array(Tuple(value_0_level_1 String, value_1_level_1 String)) +toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String +toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String +[('value_0_level_1','value_1_level_1')] [\'value_0_level_1\'] [\'value_1_level_1\'] +-- +tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(String) +tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(String) +['value_0_level_1'] ['value_1_level_1'] +-- +toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String +toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String +[\'value_0_level_1\'] [\'value_1_level_1\'] +Nested +id UInt64 +value.value_0_level_0 Array(Nested(value_0_level_1 String, value_1_level_1 String)) +value.value_1_level_0 Array(String) +0 [[('value_0_level_1','value_1_level_1')]] ['value_1_level_0'] +-- +value.value_0_level_0 Array(Nested(value_0_level_1 String, value_1_level_1 String)) +value.value_1_level_0 Array(String) +[[('value_0_level_1','value_1_level_1')]] ['value_1_level_0'] +-- +value.value_0_level_0.value_0_level_1 Array(Array(String)) +value.value_0_level_0.value_1_level_1 Array(Array(String)) +[['value_0_level_1']] [['value_1_level_1']] +-- +value.value_0_level_0 Array(Nested(value_0_level_1 String, value_1_level_1 String)) +tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(Array(String)) +tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(Array(String)) +[[('value_0_level_1','value_1_level_1')]] [['value_0_level_1']] [['value_1_level_1']] +-- +value.value_0_level_0 Array(Nested(value_0_level_1 String, value_1_level_1 String)) +tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(Array(String)) +tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(Array(String)) +[[('value_0_level_1','value_1_level_1')]] [['value_0_level_1']] [['value_1_level_1']] +-- +value.value_0_level_0 Array(Nested(value_0_level_1 String, value_1_level_1 String)) +toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String +toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String +[[('value_0_level_1','value_1_level_1')]] [[\'value_0_level_1\']] [[\'value_1_level_1\']] +-- +tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(Array(String)) +tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(Array(String)) +[['value_0_level_1']] [['value_1_level_1']] +-- +toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String +toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String +[[\'value_0_level_1\']] [[\'value_1_level_1\']] diff --git a/tests/queries/0_stateless/02342_analyzer_compound_types.sql b/tests/queries/0_stateless/02342_analyzer_compound_types.sql new file mode 100644 index 00000000000..abe43ce101d --- /dev/null +++ b/tests/queries/0_stateless/02342_analyzer_compound_types.sql @@ -0,0 +1,195 @@ +SET use_analyzer = 1; + +SELECT 'Constant tuple'; + +SELECT cast((1, 'Value'), 'Tuple (id UInt64, value String)') AS value, value.id, value.value; +SELECT cast((1, 'Value'), 'Tuple (id UInt64, value String)') AS value, value.* APPLY toString; +SELECT cast((1, 'Value'), 'Tuple (id UInt64, value String)') AS value, value.COLUMNS(id) APPLY toString; +SELECT cast((1, 'Value'), 'Tuple (id UInt64, value String)') AS value, value.COLUMNS(value) APPLY toString; +SELECT cast((1, 'Value'), 'Tuple (id UInt64, value String)') AS value, value.COLUMNS('i') APPLY toString; +SELECT cast((1, 'Value'), 'Tuple (id UInt64, value String)') AS value, value.COLUMNS('v') APPLY toString; + +SELECT 'Tuple'; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, (('value_0_level_1', 'value_1_level_1'), 'value_1_level_0')); + +SELECT '--'; + +DESCRIBE (SELECT * FROM test_table); +SELECT * FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT id, value FROM test_table); +SELECT id, value FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0, value.value_1_level_0 FROM test_table); +SELECT value.value_0_level_0, value.value_1_level_0 FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value AS alias_value, alias_value.value_0_level_0, alias_value.value_1_level_0 FROM test_table); +SELECT value AS alias_value, alias_value.value_0_level_0, alias_value.value_1_level_0 FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value AS alias_value, alias_value.* FROM test_table); +SELECT value AS alias_value, alias_value.* FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value AS alias_value, alias_value.* APPLY toString FROM test_table); +SELECT value AS alias_value, alias_value.* APPLY toString FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.* FROM test_table); +SELECT value.* FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.* APPLY toString FROM test_table); +SELECT value.* APPLY toString FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0.value_0_level_1, value.value_0_level_0.value_1_level_1 FROM test_table); +SELECT value.value_0_level_0.value_0_level_1, value.value_0_level_0.value_1_level_1 FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0 AS alias_value, alias_value.value_0_level_1, alias_value.value_1_level_1 FROM test_table); +SELECT value.value_0_level_0 AS alias_value, alias_value.value_0_level_1, alias_value.value_1_level_1 FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0 AS alias_value, alias_value.* FROM test_table); +SELECT value.value_0_level_0 AS alias_value, alias_value.* FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0 AS alias_value, alias_value.* APPLY toString FROM test_table); +SELECT value.value_0_level_0 AS alias_value, alias_value.* APPLY toString FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0.* FROM test_table); +SELECT value.value_0_level_0.* FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0.* APPLY toString FROM test_table); +SELECT value.value_0_level_0.* APPLY toString FROM test_table; + +DROP TABLE test_table; + +SELECT 'Array of tuples'; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value Array(Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String)) +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, [('value_0_level_1', 'value_1_level_1')], ['value_1_level_0']); + +DESCRIBE (SELECT * FROM test_table); +SELECT * FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0, value.value_1_level_0 FROM test_table); +SELECT value.value_0_level_0, value.value_1_level_0 FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0.value_0_level_1, value.value_0_level_0.value_1_level_1 FROM test_table); +SELECT value.value_0_level_0.value_0_level_1, value.value_0_level_0.value_1_level_1 FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0 AS alias_value, alias_value.value_0_level_1, alias_value.value_1_level_1 FROM test_table); +SELECT value.value_0_level_0 AS alias_value, alias_value.value_0_level_1, alias_value.value_1_level_1 FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0 AS alias_value, alias_value.* FROM test_table); +SELECT value.value_0_level_0 AS alias_value, alias_value.* FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0 AS alias_value, alias_value.* APPLY toString FROM test_table); +SELECT value.value_0_level_0 AS alias_value, alias_value.* APPLY toString FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0.* FROM test_table); +SELECT value.value_0_level_0.* FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0.* APPLY toString FROM test_table); +SELECT value.value_0_level_0.* APPLY toString FROM test_table; + +DROP TABLE test_table; + +SELECT 'Nested'; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value Nested (value_0_level_0 Nested(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, [[('value_0_level_1', 'value_1_level_1')]], ['value_1_level_0']); + +DESCRIBE (SELECT * FROM test_table); +SELECT * FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0, value.value_1_level_0 FROM test_table); +SELECT value.value_0_level_0, value.value_1_level_0 FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0.value_0_level_1, value.value_0_level_0.value_1_level_1 FROM test_table); +SELECT value.value_0_level_0.value_0_level_1, value.value_0_level_0.value_1_level_1 FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0 AS value_alias, value_alias.value_0_level_1, value_alias.value_1_level_1 FROM test_table); +SELECT value.value_0_level_0 AS value_alias, value_alias.value_0_level_1, value_alias.value_1_level_1 FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0 AS value_alias, value_alias.* FROM test_table); +SELECT value.value_0_level_0 AS value_alias, value_alias.* FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0 AS value_alias, value_alias.* APPLY toString FROM test_table); +SELECT value.value_0_level_0 AS value_alias, value_alias.* APPLY toString FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0.* FROM test_table); +SELECT value.value_0_level_0.* FROM test_table; + +SELECT '--'; + +DESCRIBE (SELECT value.value_0_level_0.* APPLY toString FROM test_table); +SELECT value.value_0_level_0.* APPLY toString FROM test_table; + +DROP TABLE test_table; From 9d4c508b39bc7479874fcb1c839cf1ff5065618f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Jul 2022 14:06:25 +0200 Subject: [PATCH 052/188] Added lambda tests --- src/Analyzer/ColumnNode.cpp | 4 ++ src/Analyzer/QueryAnalysisPass.cpp | 39 +++++++----- .../02343_analyzer_lambdas.reference | 25 ++++++++ .../0_stateless/02343_analyzer_lambdas.sql | 60 +++++++++++++++++++ 4 files changed, 114 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/02343_analyzer_lambdas.reference create mode 100644 tests/queries/0_stateless/02343_analyzer_lambdas.sql diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index 3b112cf0909..ac8d5200815 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -42,6 +42,10 @@ void ColumnNode::updateTreeHashImpl(HashState & hash_state) const hash_state.update(column.name.size()); hash_state.update(column.name); + const auto & column_type_name = column.type->getName(); + hash_state.update(column_type_name.size()); + hash_state.update(column_type_name); + auto column_source_ptr = column_source.lock(); if (column_source_ptr) column_source_ptr->updateTreeHashImpl(hash_state); diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 56c43c7223d..818c0b66f5f 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -147,12 +147,16 @@ namespace ErrorCodes * * TODO: Add expression name into query tree node. Example: SELECT plus(1, 1). Result: SELECT 2. Expression name of constant node should be 2. * TODO: Table identifiers with optional UUID. + * TODO: Table ALIAS columns * TODO: Support STRICT except, replace matchers. * TODO: Support multiple entities with same alias. * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. - * TODO: CTE, JOIN, ARRAY JOIN, bulding sets, grouping, in. + * TODO: CTE + * TODO: JOIN, ARRAY JOIN + * TODO: bulding sets + * TODO: Special functions grouping, in. */ /// Identifier lookup context @@ -1222,6 +1226,9 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, for (const auto & element_name : element_names) { + if (!matcher_node_typed.isMatchingColumn(element_name)) + continue; + auto tuple_element_function = std::make_shared("tupleElement"); tuple_element_function->getArguments().getNodes().push_back(expression_query_tree_node); tuple_element_function->getArguments().getNodes().push_back(std::make_shared(element_name)); @@ -1259,10 +1266,14 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, */ IQueryTreeNode * scope_node = scope.scope_node.get(); auto * scope_query_node = scope_node->as(); - while (scope.scope_node && !scope_query_node) + + while (!scope_query_node) { - if (scope.parent_scope) - scope_node = scope.parent_scope->scope_node.get(); + if (!scope.parent_scope) + break; + + scope_node = scope.parent_scope->scope_node.get(); + scope_query_node = scope_node->as(); } /// If there are no parent scope that has tables or query scope does not have FROM section @@ -1683,11 +1694,11 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc for (auto & function_lambda_argument_index : function_lambda_arguments_indexes) { - auto lambda_argument_clone = function_arguments[function_lambda_argument_index]->clone(); - auto & lambda_argument_clone_typed = lambda_argument_clone->as(); - const auto & lambda_argument_clone_argument_names = lambda_argument_clone_typed.getArgumentNames(); + auto lambda_to_resolve = function_arguments[function_lambda_argument_index]->clone(); + auto & lambda_to_resolve_typed = lambda_to_resolve->as(); - size_t lambda_arguments_size = lambda_argument_clone_typed.getArguments().getNodes().size(); + const auto & lambda_argument_names = lambda_to_resolve_typed.getArgumentNames(); + size_t lambda_arguments_size = lambda_to_resolve_typed.getArguments().getNodes().size(); const auto * function_data_type = typeid_cast(argument_types[function_lambda_argument_index].get()); if (!function_data_type) @@ -1715,16 +1726,16 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc for (size_t i = 0; i < lambda_arguments_size; ++i) { const auto & argument_type = function_data_type_argument_types[i]; - auto column_name_and_type = NameAndTypePair{lambda_argument_clone_argument_names[i], argument_type}; - lambda_arguments.push_back(std::make_shared(std::move(column_name_and_type), lambda_argument_clone)); + auto column_name_and_type = NameAndTypePair{lambda_argument_names[i], argument_type}; + lambda_arguments.push_back(std::make_shared(std::move(column_name_and_type), lambda_to_resolve)); } - IdentifierResolveScope lambda_scope(lambda_argument_clone, &scope /*parent_scope*/); - resolveLambda(lambda_argument_clone, lambda_arguments, lambda_scope); + IdentifierResolveScope lambda_scope(lambda_to_resolve, &scope /*parent_scope*/); + resolveLambda(lambda_to_resolve, lambda_arguments, lambda_scope); - argument_types[function_lambda_argument_index] = std::make_shared(function_data_type_argument_types, lambda_argument_clone->getResultType()); + argument_types[function_lambda_argument_index] = std::make_shared(function_data_type_argument_types, lambda_to_resolve->getResultType()); argument_columns[function_lambda_argument_index].type = argument_types[function_lambda_argument_index]; - function_arguments[function_lambda_argument_index] = std::move(lambda_argument_clone); + function_arguments[function_lambda_argument_index] = std::move(lambda_to_resolve); } } diff --git a/tests/queries/0_stateless/02343_analyzer_lambdas.reference b/tests/queries/0_stateless/02343_analyzer_lambdas.reference new file mode 100644 index 00000000000..4bfd09df731 --- /dev/null +++ b/tests/queries/0_stateless/02343_analyzer_lambdas.reference @@ -0,0 +1,25 @@ +Standalone lambdas +2 +1 \N [1,2,3] +1 \N [1,2,3] +1 +0 Value +Lambda as function parameter +[2,3,4] +[2,3,4] +['1','2','3'] ['1','2','3'] +['1','2','3'] ['1','2','3'] +[0,0,0] +[1,2,3] +['1_0','2_0','3_0'] +Lambda compound argument +(1,'Value') 1_Value +value_0_level_0_value_1_level_0 +Lambda matcher +0 +0 Value +Lambda untuple +(1,'Value') 1 Value +Lambda carrying +2 1 +1 0 diff --git a/tests/queries/0_stateless/02343_analyzer_lambdas.sql b/tests/queries/0_stateless/02343_analyzer_lambdas.sql new file mode 100644 index 00000000000..10623bdaa34 --- /dev/null +++ b/tests/queries/0_stateless/02343_analyzer_lambdas.sql @@ -0,0 +1,60 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +SELECT 'Standalone lambdas'; + +WITH x -> x + 1 AS lambda SELECT lambda(1); +WITH x -> toString(x) AS lambda SELECT lambda(1), lambda(NULL), lambda([1,2,3]); +WITH x -> toString(x) AS lambda_1, lambda_1 AS lambda_2, lambda_2 AS lambda_3 SELECT lambda_1(1), lambda_2(NULL), lambda_3([1,2,3]); + +WITH x -> x + 1 AS lambda SELECT lambda(id) FROM test_table; +WITH x -> toString(x) AS lambda SELECT lambda(id), lambda(value) FROM test_table; + +SELECT 'Lambda as function parameter'; + +SELECT arrayMap(x -> x + 1, [1,2,3]); +WITH x -> x + 1 AS lambda SELECT arrayMap(lambda, [1,2,3]); +SELECT arrayMap((x -> toString(x)) as lambda, [1,2,3]), arrayMap(lambda, ['1','2','3']); +WITH x -> toString(x) AS lambda_1 SELECT arrayMap(lambda_1 AS lambda_2, [1,2,3]), arrayMap(lambda_2, ['1', '2', '3']); + +SELECT arrayMap(x -> id, [1,2,3]) FROM test_table; +SELECT arrayMap(x -> x + id, [1,2,3]) FROM test_table; +SELECT arrayMap((x -> concat(concat(toString(x), '_'), toString(id))) as lambda, [1,2,3]) FROM test_table; + +SELECT 'Lambda compound argument'; + +DROP TABLE IF EXISTS test_table_tuple; +CREATE TABLE test_table_tuple +( + id UInt64, + value Tuple(value_0_level_0 String, value_1_level_0 String) +) ENGINE=TinyLog; + +INSERT INTO test_table_tuple VALUES (0, ('value_0_level_0', 'value_1_level_0')); + +WITH x -> concat(concat(toString(x.id), '_'), x.value) AS lambda SELECT cast((1, 'Value'), 'Tuple (id UInt64, value String)') AS value, lambda(value); +WITH x -> concat(concat(x.value_0_level_0, '_'), x.value_1_level_0) AS lambda SELECT lambda(value) FROM test_table_tuple; + +SELECT 'Lambda matcher'; + +WITH x -> * AS lambda SELECT lambda(1); +WITH x -> * AS lambda SELECT lambda(1) FROM test_table; + +SELECT 'Lambda untuple'; + +WITH x -> untuple(x) AS lambda SELECT cast((1, 'Value'), 'Tuple (id UInt64, value String)') AS value, lambda(value); + +SELECT 'Lambda carrying'; + +WITH functor, x -> functor(x) AS lambda, x -> x + 1 AS functor_1, x -> toString(x) AS functor_2 SELECT lambda(functor_1, 1), lambda(functor_2, 1); +WITH functor, x -> functor(x) AS lambda, x -> x + 1 AS functor_1, x -> toString(x) AS functor_2 SELECT lambda(functor_1, id), lambda(functor_2, id) FROM test_table; + +DROP TABLE test_table; From d835f70fdcbd4763775e6a226dda55f71412515e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Jul 2022 18:02:47 +0200 Subject: [PATCH 053/188] Added support for STRICT column transformers --- src/Analyzer/ColumnTransformers.cpp | 6 +- src/Analyzer/ColumnTransformers.h | 51 +++++++++- src/Analyzer/QueryAnalysisPass.cpp | 93 ++++++++++++++++++- src/Analyzer/QueryTreeBuilder.cpp | 4 +- ...lyzer_column_transformers_strict.reference | 1 + ...43_analyzer_column_transformers_strict.sql | 18 ++++ 6 files changed, 162 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02343_analyzer_column_transformers_strict.reference create mode 100644 tests/queries/0_stateless/02343_analyzer_column_transformers_strict.sql diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp index 7d51cb907b8..db889a0e6ba 100644 --- a/src/Analyzer/ColumnTransformers.cpp +++ b/src/Analyzer/ColumnTransformers.cpp @@ -191,12 +191,13 @@ QueryTreeNodePtr ExceptColumnTransformerNode::cloneImpl() const if (except_transformer_type == ExceptColumnTransformerType::REGEXP) return std::make_shared(column_matcher); - return std::make_shared(except_column_names); + return std::make_shared(except_column_names, is_strict); } /// ReplaceColumnTransformerNode implementation -ReplaceColumnTransformerNode::ReplaceColumnTransformerNode(const std::vector & replacements_) +ReplaceColumnTransformerNode::ReplaceColumnTransformerNode(const std::vector & replacements_, bool is_strict_) + : is_strict(is_strict_) { children.resize(1); children[replacements_child_index] = std::make_shared(); @@ -294,6 +295,7 @@ QueryTreeNodePtr ReplaceColumnTransformerNode::cloneImpl() const { ReplaceColumnTransformerNodePtr result_replace_transformers(new ReplaceColumnTransformerNode()); + result_replace_transformers->is_strict = is_strict; result_replace_transformers->replacements_names = replacements_names; return result_replace_transformers; diff --git a/src/Analyzer/ColumnTransformers.h b/src/Analyzer/ColumnTransformers.h index 23c7a132271..789c17ffcec 100644 --- a/src/Analyzer/ColumnTransformers.h +++ b/src/Analyzer/ColumnTransformers.h @@ -157,13 +157,23 @@ const char * toString(ExceptColumnTransformerType type); class ExceptColumnTransformerNode; using ExceptColumnTransformerNodePtr = std::shared_ptr; -/// Except column transformer +/** Except column transformer + * Strict column transformer must use all column names during matched nodes transformation. + * + * Example: + * CREATE TABLE test_table (id UInt64, value String) ENGINE=TinyLog; + * SELECT * EXCEPT STRICT (id, value1) FROM test_table; + * Such query will throw exception because column name with value1 was not matched by strict EXCEPT transformer. + * + * Strict is valid only for EXCEPT COLUMN_LIST transformer. + */ class ExceptColumnTransformerNode final : public IColumnTransformerNode { public: /// Initialize except column transformer with column names - explicit ExceptColumnTransformerNode(Names except_column_names_) + explicit ExceptColumnTransformerNode(Names except_column_names_, bool is_strict_) : except_transformer_type(ExceptColumnTransformerType::COLUMN_LIST) + , is_strict(is_strict_) , except_column_names(std::move(except_column_names_)) { } @@ -171,6 +181,7 @@ public: /// Initialize except column transformer with regexp column matcher explicit ExceptColumnTransformerNode(std::shared_ptr column_matcher_) : except_transformer_type(ExceptColumnTransformerType::REGEXP) + , is_strict(false) , column_matcher(std::move(column_matcher_)) { } @@ -181,9 +192,25 @@ public: return except_transformer_type; } + /** Get is except transformer strict. + * Valid only for EXCEPT COLUMN_LIST transformer. + */ + bool isStrict() const + { + return is_strict; + } + /// Returns true if except transformer match column name, false otherwise. bool isColumnMatching(const std::string & column_name) const; + /** Get except column names. + * Valid only for column list except transformer. + */ + const Names & getExceptColumnNames() const + { + return except_column_names; + } + ColumnTransfomerType getTransformerType() const override { return ColumnTransfomerType::EXCEPT; @@ -197,8 +224,10 @@ protected: ASTPtr toASTImpl() const override; QueryTreeNodePtr cloneImpl() const override; + private: ExceptColumnTransformerType except_transformer_type; + bool is_strict; Names except_column_names; std::shared_ptr column_matcher; }; @@ -206,7 +235,14 @@ private: class ReplaceColumnTransformerNode; using ReplaceColumnTransformerNodePtr = std::shared_ptr; -/// Replace column transformer +/** Replace column transformer + * Strict replace column transformer must use all replacements during matched nodes transformation. + * + * Example: + * REATE TABLE test_table (id UInt64, value String) ENGINE=TinyLog; + * SELECT * REPLACE STRICT (1 AS id, 2 AS value_1) FROM test_table; + * Such query will throw exception because column name with value1 was not matched by strict REPLACE transformer. + */ class ReplaceColumnTransformerNode final : public IColumnTransformerNode { public: @@ -218,13 +254,19 @@ public: }; /// Initialize replace column transformer with replacements - explicit ReplaceColumnTransformerNode(const std::vector & replacements_); + explicit ReplaceColumnTransformerNode(const std::vector & replacements_, bool is_strict); ColumnTransfomerType getTransformerType() const override { return ColumnTransfomerType::REPLACE; } + /// Is replace column transformer strict + bool isStrict() const + { + return is_strict; + } + /// Get replacements ListNode & getReplacements() const { @@ -260,6 +302,7 @@ protected: private: ReplaceColumnTransformerNode() = default; + bool is_strict; Names replacements_names; static constexpr size_t replacements_child_index = 0; }; diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 818c0b66f5f..87c4090cf4d 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -148,7 +148,6 @@ namespace ErrorCodes * TODO: Add expression name into query tree node. Example: SELECT plus(1, 1). Result: SELECT 2. Expression name of constant node should be 2. * TODO: Table identifiers with optional UUID. * TODO: Table ALIAS columns - * TODO: Support STRICT except, replace matchers. * TODO: Support multiple entities with same alias. * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. @@ -1185,7 +1184,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifier(const IdentifierLookup & id /// Resolve query tree nodes functions implementation -/** Resolve query tree matcher. Check MatcherNode.h for detailed matcher description. +/** Resolve query tree matcher. Check MatcherNode.h for detailed matcher description. Check ColumnTransformers.h for detailed transformers description. * * 1. Populate matcher_expression_nodes. * @@ -1195,7 +1194,8 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifier(const IdentifierLookup & id * * If we resolve non qualified matcher, use current scope join tree node. * - * 2. Apply column transforms to matched expression nodes. + * 2. Apply column transformers to matched expression nodes. For strict column transformers save used column names. + * 3. Validate strict column transformers. */ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope) { @@ -1316,7 +1316,21 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, } } + std::unordered_map> strict_transformer_to_used_column_names; + auto add_strict_transformer_column_name = [&](const IColumnTransformerNode * transformer, const std::string & column_name) + { + auto it = strict_transformer_to_used_column_names.find(transformer); + if (it == strict_transformer_to_used_column_names.end()) + { + auto [inserted_it, _] = strict_transformer_to_used_column_names.emplace(transformer, std::unordered_set()); + it = inserted_it; + } + + it->second.insert(column_name); + }; + ListNodePtr list = std::make_shared(); + for (auto & node : matcher_expression_nodes) { for (const auto & transformer : matcher_node_typed.getColumnTransformers().getNodes()) @@ -1352,8 +1366,12 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, else if (auto * except_transformer = transformer->as()) { auto node_name = node->getName(); + if (except_transformer->isColumnMatching(node_name)) { + if (except_transformer->isStrict()) + add_strict_transformer_column_name(except_transformer, node_name); + node = {}; break; } @@ -1365,6 +1383,9 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, if (!replace_expression) continue; + if (replace_transformer->isStrict()) + add_strict_transformer_column_name(replace_transformer, node_name); + node = replace_expression; resolveExpressionNode(node, scope, false /*allow_lambda_expression*/); } @@ -1374,6 +1395,72 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, list->getNodes().push_back(node); } + for (auto & [strict_transformer, used_column_names] : strict_transformer_to_used_column_names) + { + auto strict_transformer_type = strict_transformer->getTransformerType(); + const Names * strict_transformer_column_names = nullptr; + + switch (strict_transformer_type) + { + case ColumnTransfomerType::EXCEPT: + { + const auto * except_transformer = static_cast(strict_transformer); + const auto & except_names = except_transformer->getExceptColumnNames(); + + if (except_names.size() != used_column_names.size()) + strict_transformer_column_names = &except_transformer->getExceptColumnNames(); + + break; + } + case ColumnTransfomerType::REPLACE: + { + const auto * replace_transformer = static_cast(strict_transformer); + const auto & replacement_names = replace_transformer->getReplacementsNames(); + + if (replacement_names.size() != used_column_names.size()) + strict_transformer_column_names = &replace_transformer->getReplacementsNames(); + + break; + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected strict EXCEPT or REPLACE column transformer. Actual type {}. In scope {}", + toString(strict_transformer_type), + scope.scope_node->formatASTForErrorMessage()); + } + } + + if (!strict_transformer_column_names) + continue; + + Names non_matched_column_names; + size_t strict_transformer_column_names_size = strict_transformer_column_names->size(); + for (size_t i = 0; i < strict_transformer_column_names_size; ++i) + { + const auto & column_name = (*strict_transformer_column_names)[i]; + if (used_column_names.find(column_name) == used_column_names.end()) + non_matched_column_names.push_back(column_name); + } + + WriteBufferFromOwnString non_matched_column_names_buffer; + size_t non_matched_column_names_size = non_matched_column_names.size(); + for (size_t i = 0; i < non_matched_column_names_size; ++i) + { + const auto & column_name = non_matched_column_names[i]; + + non_matched_column_names_buffer << column_name; + if (i + 1 != non_matched_column_names_size) + non_matched_column_names_buffer << ", "; + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Strict {} column transformer {} expects following column(s) {}", + toString(strict_transformer_type), + strict_transformer->formatASTForErrorMessage(), + non_matched_column_names_buffer.str()); + } + return list; } diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 3c041635dad..b5d1ad837ed 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -480,7 +480,7 @@ ColumnTransformersNodes QueryTreeBuilder::getColumnTransformers(const ASTPtr & m for (auto & except_transformer_child : except_transformer->children) except_column_names.push_back(except_transformer_child->as().full_name); - column_transformers.emplace_back(std::make_shared(std::move(except_column_names))); + column_transformers.emplace_back(std::make_shared(std::move(except_column_names), except_transformer->is_strict)); } } else if (auto * replace_transformer = child->as()) @@ -494,7 +494,7 @@ ColumnTransformersNodes QueryTreeBuilder::getColumnTransformers(const ASTPtr & m replacements.emplace_back(ReplaceColumnTransformerNode::Replacement{replacement.name, getExpression(replacement.expr)}); } - column_transformers.emplace_back(std::make_shared(replacements)); + column_transformers.emplace_back(std::make_shared(replacements, replace_transformer->is_strict)); } else { diff --git a/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.reference b/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.reference new file mode 100644 index 00000000000..af53c9c3c21 --- /dev/null +++ b/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.reference @@ -0,0 +1 @@ +Value diff --git a/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.sql b/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.sql new file mode 100644 index 00000000000..657e2098b4a --- /dev/null +++ b/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.sql @@ -0,0 +1,18 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +SELECT * EXCEPT (id) FROM test_table; +SELECT * EXCEPT STRICT (id, value1) FROM test_table; -- { serverError 36 } + +SELECT * REPLACE STRICT (1 AS id, 2 AS value) FROM test_table; +SELECT * REPLACE STRICT (1 AS id, 2 AS value_1) FROM test_table; -- { serverError 36 } + +DROP TABLE IF EXISTS test_table; From 2d61c45f9438abeee541ef6c9df2aa353ea8c7f5 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Jul 2022 13:09:28 +0200 Subject: [PATCH 054/188] Added support for multiple expressions with same alias --- src/Analyzer/ColumnTransformers.cpp | 2 + src/Analyzer/IQueryTreeNode.cpp | 3 + src/Analyzer/QueryAnalysisPass.cpp | 167 ++++++++++++------ ..._multiple_aliases_for_expression.reference | 4 + ...alyzer_multiple_aliases_for_expression.sql | 26 +++ 5 files changed, 146 insertions(+), 56 deletions(-) create mode 100644 tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.reference create mode 100644 tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.sql diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp index db889a0e6ba..78c2becb0fa 100644 --- a/src/Analyzer/ColumnTransformers.cpp +++ b/src/Analyzer/ColumnTransformers.cpp @@ -72,6 +72,8 @@ void ApplyColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) c void ApplyColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const { hash_state.update(static_cast(getTransformerType())); + hash_state.update(static_cast(getApplyTransformerType())); + getExpressionNode()->updateTreeHash(hash_state); } diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 8a84ad2a9df..e5d47b98117 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -55,7 +55,10 @@ IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const void IQueryTreeNode::updateTreeHash(HashState & state) const { + state.update(static_cast(getNodeType())); + updateTreeHashImpl(state); + state.update(children.size()); for (const auto & child : children) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 87c4090cf4d..f4f1856e37f 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -67,6 +67,7 @@ namespace ErrorCodes extern const int CYCLIC_ALIASES; extern const int INCORRECT_RESULT_OF_SCALAR_SUBQUERY; extern const int BAD_ARGUMENTS; + extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -148,7 +149,6 @@ namespace ErrorCodes * TODO: Add expression name into query tree node. Example: SELECT plus(1, 1). Result: SELECT 2. Expression name of constant node should be 2. * TODO: Table identifiers with optional UUID. * TODO: Table ALIAS columns - * TODO: Support multiple entities with same alias. * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. @@ -260,12 +260,7 @@ public: if (node->hasAlias()) { expressions.emplace_back(node.get(), node->getAlias()); - - auto [it, inserted] = expressions_aliases.emplace(expressions.back().second); - if (!inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expression with alias {} already exists in stack", - node->getAlias()); + ++alias_name_to_expressions_size[expressions.back().second]; return; } @@ -276,7 +271,13 @@ public: { const auto & [_, top_expression_alias] = expressions.back(); if (!top_expression_alias.empty()) - expressions_aliases.erase(top_expression_alias); + { + auto it = alias_name_to_expressions_size.find(top_expression_alias); + --it->second; + + if (it->second == 0) + alias_name_to_expressions_size.erase(it); + } expressions.pop_back(); } @@ -299,7 +300,7 @@ public: bool hasExpressionWithAlias(const std::string & alias) const { - return expressions_aliases.find(alias) != expressions_aliases.end(); + return alias_name_to_expressions_size.find(alias) != alias_name_to_expressions_size.end(); } size_t size() const @@ -330,7 +331,7 @@ public: private: std::vector> expressions; - std::unordered_set expressions_aliases; + std::unordered_map alias_name_to_expressions_size; }; struct IdentifierResolveScope @@ -358,6 +359,9 @@ struct IdentifierResolveScope /// Alias name to table expression node std::unordered_map alias_name_to_table_expression_node; + /// Nodes with duplicated identifiers + std::unordered_set nodes_with_duplicated_aliases; + /// Current scope expression in resolve process stack ExpressionsStack expressions_in_resolve_process_stack; @@ -391,6 +395,10 @@ struct IdentifierResolveScope for (const auto & [alias_name, node] : alias_name_to_table_expression_node) buffer << "Alias name " << alias_name << " table node " << node->formatASTForErrorMessage() << '\n'; + buffer << "Nodes with duplicated aliases " << nodes_with_duplicated_aliases.size() << '\n'; + for (const auto & node : nodes_with_duplicated_aliases) + buffer << "Alias name " << node->getAlias() << " node " << node->formatASTForErrorMessage() << '\n'; + buffer << "Expression resolve process stack " << '\n'; expressions_in_resolve_process_stack.dump(buffer); @@ -641,7 +649,9 @@ void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node) scalar_type = std::make_shared(block.getDataTypes()); } + auto original_ast = node->getOriginalAST(); node = std::make_shared(std::move(scalar_value), std::move(scalar_type)); + node->setOriginalAST(std::move(original_ast)); } /// Resolve identifier functions implementation @@ -690,24 +700,24 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(cons /** Visitor that extracts expression and function aliases from node and initialize scope tables with it. * Does not go into child lambdas and queries. - * If there are multiple entities with same alias, exception is raised. * * TODO: Maybe better for this visitor to handle QueryNode. Handle table nodes. * * Important: * Identifier nodes with aliases are added both in alias to expression and alias to function map. * - * These is necessary because identifier with alias can give any node alias name, expression or function. - * - * TODO: Disable identifier with alias node propagation for table nodes. This can occur only for special functions - * if their argument can be table. + * These is necessary because identifier with alias can give alias name to any query tree node. * * Example: * WITH (x -> x + 1) AS id, id AS value SELECT value(1); * In this example id as value is identifier node that has alias, during scope initialization we cannot derive * that id is actually lambda or expression. * - * There are no easy solution here, without trying to make full featured expression resolution at this stage, because example can be much complex: + * + * TODO: Disable identifier with alias node propagation for table nodes. This can occur only for special functions + * if their argument can be table. + * + * There are no easy solution here, without trying to make full featured expression resolution at this stage. * Example: * WITH (x -> x + 1) AS id, id AS id_1, id_1 AS id_2 SELECT id_2(1); * @@ -754,32 +764,25 @@ private: return; const auto & alias = node->getAlias(); - auto throw_exception = [&]() - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Having multiple alises with same name {} is not allowed. In scope {}", - alias, - data.scope.scope_node->formatASTForErrorMessage()); - }; if (function_node) { if (data.scope.alias_name_to_expression_node.contains(alias)) - throw_exception(); + data.scope.nodes_with_duplicated_aliases.insert(node); auto [_, inserted] = data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); if (!inserted) - throw_exception(); + data.scope.nodes_with_duplicated_aliases.insert(node); return; } if (data.scope.alias_name_to_lambda_node.contains(alias)) - throw_exception(); + data.scope.nodes_with_duplicated_aliases.insert(node); auto [_, inserted] = data.scope.alias_name_to_expression_node.insert(std::make_pair(alias, node)); if (!inserted) - throw_exception(); + data.scope.nodes_with_duplicated_aliases.insert(node); /// If node is identifier put it also in scope alias name to lambda node map if (node->getNodeType() == QueryTreeNodeType::IDENTIFIER) @@ -827,14 +830,14 @@ using ScopeAliasVisitor = ScopeAliasVisitorMatcher::Visitor; * * Special case for QueryNode, if lookup context is expression, evaluate it as scalar subquery. * - * 6. If identifier is compound and identifier lookup is in expression context, pop first part from identifier lookup and wrap alias node + * 6. Pop node from current expressions to resolve. + * 7. If identifier is compound and identifier lookup is in expression context, pop first part from identifier lookup and wrap alias node * using nested parts of identifier using `wrapExpressionNodeInTupleElement` function. * * Example: SELECT value AS alias, alias.nested_path. * Result: SELECT value AS alias, tupleElement(value, 'nested_path') value.nested_path. * - * 7. If identifier lookup is in expression context, clone result expression. - * 8. Pop node from current expressions to resolve. + * 8. If identifier lookup is in expression context, clone result expression. */ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) { @@ -918,25 +921,21 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier evaluateScalarSubquery(it->second); } - if (!it->second) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Node with alias {} is not valid. In scope {}", - identifier_bind_part, - scope.scope_node->formatASTForErrorMessage()); + scope.expressions_in_resolve_process_stack.popNode(); QueryTreeNodePtr result = it->second; - if (identifier_lookup.identifier.isCompound() && identifier_lookup.isExpressionLookup()) + if (identifier_lookup.identifier.isCompound() && identifier_lookup.isExpressionLookup() && result) { auto nested_path = IdentifierView(identifier_lookup.identifier); nested_path.popFirst(); - auto tuple_element_result = wrapExpressionNodeInTupleElement(it->second, nested_path); + auto tuple_element_result = wrapExpressionNodeInTupleElement(result, nested_path); resolveFunction(tuple_element_result, scope); result = tuple_element_result; } - else if (identifier_lookup.isExpressionLookup()) + else if (identifier_lookup.isExpressionLookup() && result) { /** If expression node was resolved throught aliases we must clone it to keep query tree state valid. * @@ -946,12 +945,10 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier * * This query is broken because multiple aliases with same name are not allowed in query tree. */ - result = it->second->clone(); + result = result->clone(); result->removeAlias(); } - scope.expressions_in_resolve_process_stack.popNode(); - return result; } @@ -1858,7 +1855,10 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc /// Replace function node with result constant node Field constant_value; column->get(0, constant_value); + + auto original_ast = function_node.getOriginalAST(); node = std::make_shared(std::move(constant_value), result_type); + node->setOriginalAST(std::move(original_ast)); return; } } @@ -1892,16 +1892,16 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc */ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression) { - String node_alias; + String node_alias = node->getAlias(); - if (node->hasAlias()) + /// Do not update node from alias table if we resolve it for duplicate alias + if (!node_alias.empty() && scope.nodes_with_duplicated_aliases.contains(node)) { /** Node could be potentially resolved by resolving other nodes. * SELECT b, a as b FROM test_table; * * To resolve b we need to resolve a. */ - node_alias = node->getAlias(); auto it = scope.alias_name_to_expression_node.find(node_alias); if (it != scope.alias_name_to_expression_node.end()) node = it->second; @@ -1996,8 +1996,10 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes scope.scope_node->formatASTForErrorMessage()); } - /// Update aliases after expression node was resolved - if (!node_alias.empty()) + /** Update aliases after expression node was resolved. + * Do not update node in alias table if we resolve it for duplicate alias. + */ + if (!node_alias.empty() && scope.nodes_with_duplicated_aliases.contains(node)) { auto it = scope.alias_name_to_expression_node.find(node_alias); if (it != scope.alias_name_to_expression_node.end()) @@ -2088,16 +2090,14 @@ void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_tree_node, IdentifierR { auto [_, inserted] = scope.alias_name_to_table_expression_node.insert(std::make_pair(from_node_alias, query_tree.getFrom())); if (!inserted) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Multiple alises does not point to same entity {}", from_node_alias); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Multiple aliases does not point to same entity {}", from_node_alias); } - /** Resolve query node sections. - * - * WITH section is not resolved at all and will be removed after query analysis pass. - * WITH section only can provide aliases to expressions and CTE for other sections to use. - * - * Example: WITH 1 AS constant, (x -> x + 1) AS lambda, a AS (SELECT * FROM test_table); - */ + /// Resolve query node sections. + + if (query_tree.getWithNode()) + resolveExpressionNodeList(query_tree.getWithNode(), scope, true /*allow_lambda_expression*/); + resolveExpressionNodeList(query_tree.getProjectionNode(), scope, false /*allow_lambda_expression*/); if (query_tree.getPrewhere()) @@ -2106,14 +2106,69 @@ void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_tree_node, IdentifierR if (query_tree.getWhere()) resolveExpressionNode(query_tree.getWhere(), scope, false /*allow_lambda_expression*/); - /// Remove WITH section + /** WITH section can be safely removed, because WITH section only can provide aliases to expressions + * and CTE for other sections to use. + * + * Example: WITH 1 AS constant, (x -> x + 1) AS lambda, a AS (SELECT * FROM test_table); + */ query_tree.getWithNode() = std::make_shared(); + + /** Resolve nodes with duplicate aliases. + * + * Such nodes during scope aliases collection are placed into duplicated array. + * After scope nodes are resolved, we can compare node with duplicate alias with + * node from scope alias table. + */ + for (const auto & node : scope.nodes_with_duplicated_aliases) + { + auto node_copy = node; + auto node_alias = node_copy->getAlias(); + resolveExpressionNode(node_copy, scope, true /*allow_lambda_expression*/); + + auto node_tree_hash = node->getTreeHash(); + bool has_node_in_alias_table = false; + + auto it = scope.alias_name_to_expression_node.find(node_alias); + if (it != scope.alias_name_to_expression_node.end()) + { + has_node_in_alias_table = true; + + if (it->second->getTreeHash() != node_tree_hash) + throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, + "Multiple expressions {} and {} for alias {}. In scope {}", + node_copy->formatASTForErrorMessage(), + it->second->formatASTForErrorMessage(), + node_alias, + scope.scope_node->formatASTForErrorMessage()); + } + + it = scope.alias_name_to_lambda_node.find(node_alias); + if (it != scope.alias_name_to_lambda_node.end()) + { + has_node_in_alias_table = true; + + if (it->second->getTreeHash() != node_tree_hash) + throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, + "Multiple expressions {} and {} for alias {}. In scope {}", + node_copy->formatASTForErrorMessage(), + it->second->formatASTForErrorMessage(), + node_alias, + scope.scope_node->formatASTForErrorMessage()); + } + + if (!has_node_in_alias_table) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Node {} with duplicate alias {} does not exists in alias table. In scope {}", + node_copy->formatASTForErrorMessage(), + node_alias, + scope.scope_node->formatASTForErrorMessage()); + } } void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { if (query_tree_node->getNodeType() != QueryTreeNodeType::QUERY) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "QueryAnalysis pass requires query tree node"); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "QueryAnalysis pass requires query node"); QueryAnalyzer analyzer(std::move(context)); analyzer.resolve(query_tree_node); diff --git a/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.reference b/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.reference new file mode 100644 index 00000000000..e0d1bb800d2 --- /dev/null +++ b/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.reference @@ -0,0 +1,4 @@ +1 1 +0 0 +2 +1 1 diff --git a/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.sql b/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.sql new file mode 100644 index 00000000000..057be94928f --- /dev/null +++ b/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.sql @@ -0,0 +1,26 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +SELECT 1 AS value, 1 AS value; +SELECT id AS value, id AS value FROM test_table; +WITH x -> x + 1 AS lambda, x -> x + 1 AS lambda SELECT lambda(1); +SELECT (SELECT 1) AS subquery, (SELECT 1) AS subquery; + +SELECT 1 AS value, 2 AS value; -- { serverError 179 } +SELECT plus(1, 1) AS value, 2 AS value; -- { serverError 179 } +SELECT (SELECT 1) AS subquery, 1 AS subquery; -- { serverError 179 } +WITH x -> x + 1 AS lambda SELECT (SELECT 1) AS lambda; -- { serverError 179 } +WITH x -> x + 1 AS lambda SELECT 1 AS lambda; -- { serverError 179 } +SELECT id AS value, value AS value FROM test_table; -- { serverError 179 } +SELECT id AS value_1, value AS value_1 FROM test_table; -- { serverError 179 } +SELECT id AS value, (id + 1) AS value FROM test_table; -- { serverError 179 } + +DROP TABLE test_table; From 6a5276e45e47399cd596ad8cb7b08c639cf9a0e7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Jul 2022 15:32:53 +0200 Subject: [PATCH 055/188] Updated tests --- src/Analyzer/ColumnNode.cpp | 19 +++++++++ src/Analyzer/ColumnNode.h | 2 + src/Analyzer/ColumnTransformers.cpp | 32 +++++++++++++++ src/Analyzer/ColumnTransformers.h | 6 +++ src/Analyzer/ConstantNode.cpp | 6 +++ src/Analyzer/ConstantNode.h | 2 + src/Analyzer/FunctionNode.cpp | 18 +++++++++ src/Analyzer/FunctionNode.h | 4 +- src/Analyzer/IQueryTreeNode.cpp | 39 +++++++++++++++++++ src/Analyzer/IQueryTreeNode.h | 23 ++++++++++- src/Analyzer/IdentifierNode.cpp | 6 +++ src/Analyzer/IdentifierNode.h | 2 + src/Analyzer/LambdaNode.cpp | 6 +++ src/Analyzer/LambdaNode.h | 2 + src/Analyzer/ListNode.cpp | 8 +++- src/Analyzer/ListNode.h | 2 + src/Analyzer/MatcherNode.cpp | 21 ++++++++++ src/Analyzer/MatcherNode.h | 2 + src/Analyzer/QueryAnalysisPass.cpp | 32 ++++++++------- src/Analyzer/QueryNode.cpp | 8 +++- src/Analyzer/QueryNode.h | 2 + src/Analyzer/TableNode.cpp | 6 +++ src/Analyzer/TableNode.h | 2 + src/Analyzer/tests/gtest_query_tree_node.cpp | 13 +++++-- ...alyzer_multiple_aliases_for_expression.sql | 1 + 25 files changed, 241 insertions(+), 23 deletions(-) diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index ac8d5200815..86b9f940e44 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -37,6 +37,25 @@ void ColumnNode::dumpTree(WriteBuffer & buffer, size_t indent) const writePointerHex(column_source_ptr.get(), buffer); } +bool ColumnNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + if (column != rhs_typed.column) + return false; + + auto source_ptr = column_source.lock(); + auto rhs_source_ptr = rhs_typed.column_source.lock(); + + if (!source_ptr && !rhs_source_ptr) + return true; + else if (source_ptr && !rhs_source_ptr) + return false; + else if (!source_ptr && rhs_source_ptr) + return false; + + return source_ptr->isEqualImpl(*rhs_source_ptr); +} + void ColumnNode::updateTreeHashImpl(HashState & hash_state) const { hash_state.update(column.name.size()); diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index 47f1bde48a3..8300b2485bb 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -80,6 +80,8 @@ public: } protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + void updateTreeHashImpl(HashState & hash_state) const override; ASTPtr toASTImpl() const override; diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp index 78c2becb0fa..c6a3945950b 100644 --- a/src/Analyzer/ColumnTransformers.cpp +++ b/src/Analyzer/ColumnTransformers.cpp @@ -69,6 +69,12 @@ void ApplyColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) c expression_node->dumpTree(buffer, indent + 4); } +bool ApplyColumnTransformerNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + return apply_transformer_type == rhs_typed.apply_transformer_type; +} + void ApplyColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const { hash_state.update(static_cast(getTransformerType())); @@ -152,6 +158,26 @@ void ExceptColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) } } +bool ExceptColumnTransformerNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + if (except_transformer_type != rhs_typed.except_transformer_type || + is_strict != rhs_typed.is_strict || + except_column_names != rhs_typed.except_column_names) + return false; + + const auto & rhs_column_matcher = rhs_typed.column_matcher; + + if (!column_matcher && !rhs_column_matcher) + return true; + else if (column_matcher && !rhs_column_matcher) + return false; + else if (!column_matcher && rhs_column_matcher) + return false; + + return column_matcher->pattern() == rhs_column_matcher->pattern(); +} + void ExceptColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const { hash_state.update(static_cast(getTransformerType())); @@ -256,6 +282,12 @@ void ReplaceColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) } } +bool ReplaceColumnTransformerNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + return is_strict == rhs_typed.is_strict && replacements_names == rhs_typed.replacements_names; +} + void ReplaceColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const { hash_state.update(static_cast(getTransformerType())); diff --git a/src/Analyzer/ColumnTransformers.h b/src/Analyzer/ColumnTransformers.h index 789c17ffcec..e5e82033ec8 100644 --- a/src/Analyzer/ColumnTransformers.h +++ b/src/Analyzer/ColumnTransformers.h @@ -132,6 +132,8 @@ public: void dumpTree(WriteBuffer & buffer, size_t indent) const override; protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override; ASTPtr toASTImpl() const override; @@ -219,6 +221,8 @@ public: void dumpTree(WriteBuffer & buffer, size_t indent) const override; protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override; ASTPtr toASTImpl() const override; @@ -293,6 +297,8 @@ public: void dumpTree(WriteBuffer & buffer, size_t indent) const override; protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override; ASTPtr toASTImpl() const override; diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 4e1824e17bc..a6c6189d03a 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -33,6 +33,12 @@ void ConstantNode::dumpTree(WriteBuffer & buffer, size_t indent) const buffer << ' ' << value.dump() << " : " << type->getName(); } +bool ConstantNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + return value == rhs_typed.value && value_string_dump == rhs_typed.value_string_dump && type->equals(*rhs_typed.type); +} + void ConstantNode::updateTreeHashImpl(HashState & hash_state) const { auto type_name = type->getName(); diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index 44593a002f0..ed605f03256 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -47,6 +47,8 @@ public: } protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + void updateTreeHashImpl(HashState & hash_state) const override; ASTPtr toASTImpl() const override; diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index 3f8b1a73a12..30e2a5a8dd0 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -72,6 +72,24 @@ String FunctionNode::getName() const return name; } +bool FunctionNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + if (function_name != rhs_typed.function_name || + isAggregateFunction() != rhs_typed.isAggregateFunction() || + isNonAggregateFunction() != rhs_typed.isNonAggregateFunction()) + return false; + + if (!result_type && !rhs_typed.result_type) + return true; + else if (result_type && !rhs_typed.result_type) + return false; + else if (!result_type && rhs_typed.result_type) + return false; + + return result_type->equals(*rhs_typed.result_type); +} + void FunctionNode::updateTreeHashImpl(HashState & hash_state) const { hash_state.update(function_name.size()); diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index d6215c7ebd5..657b81811df 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -118,7 +118,7 @@ public: /// Is function node resolved bool isResolved() const { - return function != nullptr || aggregate_function != nullptr; + return result_type != nullptr && (function != nullptr || aggregate_function != nullptr); } /// Is function node resolved as aggregate function @@ -162,6 +162,8 @@ public: String getName() const override; protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + void updateTreeHashImpl(HashState & hash_state) const override; ASTPtr toASTImpl() const override; diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index e5d47b98117..710ebb01561 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -42,6 +42,40 @@ String IQueryTreeNode::dumpTree() const return buff.str(); } +bool IQueryTreeNode::isEqual(const IQueryTreeNode & rhs) const +{ + if (getNodeType() != rhs.getNodeType()) + return false; + + if (alias != rhs.alias) + return false; + + if (!isEqualImpl(rhs)) + return false; + + size_t children_size = children.size(); + if (children_size != rhs.children.size()) + return false; + + for (size_t i = 0; i < children_size; ++i) + { + const auto & child = children[i]; + const auto & rhs_child = rhs.children[i]; + + if (!child && !rhs_child) + continue; + else if (child && !rhs_child) + return false; + else if (!child && rhs_child) + return false; + + if (!child->isEqual(*rhs.children[i])) + return false; + } + + return true; +} + IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const { HashState hash_state; @@ -56,6 +90,11 @@ IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const void IQueryTreeNode::updateTreeHash(HashState & state) const { state.update(static_cast(getNodeType())); + if (!alias.empty()) + { + state.update(alias.size()); + state.update(alias); + } updateTreeHashImpl(state); diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index df99e0e2b90..a126413bfab 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -92,10 +92,21 @@ public: /// Dump query tree to buffer starting with indent virtual void dumpTree(WriteBuffer & buffer, size_t indent) const = 0; + /** Is tree equal to other tree with node root. + * + * Aliases of query tree nodes are compared during isEqual call. + * Original ASTs of query tree nodes are not compared during isEqual call. + */ + bool isEqual(const IQueryTreeNode & rhs) const; + using Hash = std::pair; using HashState = SipHash; - /// Get tree hash identifying current tree + /** Get tree hash identifying current tree + * + * Alias of query tree node is part of query tree hash. + * Original AST is not part of query tree hash. + */ Hash getTreeHash() const; /// Update tree hash @@ -183,7 +194,15 @@ public: return children; } - /** Subclass must update tree hash of its internal state and do not update tree hash for children. + /** Subclass must compare its internal state with rhs node and do not compare its children with rhs node children. + * Caller must compare node and rhs node children. + * + * This method is not protected because if subclass node has weak pointers to other query tree nodes it must use it + * as part of its isEqualImpl method. In child classes this method should be protected. + */ + virtual bool isEqualImpl(const IQueryTreeNode & rhs) const = 0; + + /** Subclass must update tree hash with its internal state and do not update tree hash for children. * Caller must update tree hash for node children. * * This method is not protected because if subclass node has weak pointers to other query tree nodes it must use it diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp index 06892344dd0..510a9cec93c 100644 --- a/src/Analyzer/IdentifierNode.cpp +++ b/src/Analyzer/IdentifierNode.cpp @@ -17,6 +17,12 @@ void IdentifierNode::dumpTree(WriteBuffer & buffer, size_t indent) const buffer << ' ' << identifier.getFullName(); } +bool IdentifierNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + return identifier == rhs_typed.identifier; +} + void IdentifierNode::updateTreeHashImpl(HashState & state) const { const auto & identifier_name = identifier.getFullName(); diff --git a/src/Analyzer/IdentifierNode.h b/src/Analyzer/IdentifierNode.h index 054a8d323bb..38f909173f4 100644 --- a/src/Analyzer/IdentifierNode.h +++ b/src/Analyzer/IdentifierNode.h @@ -40,6 +40,8 @@ public: } protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + void updateTreeHashImpl(HashState & state) const override; ASTPtr toASTImpl() const override; diff --git a/src/Analyzer/LambdaNode.cpp b/src/Analyzer/LambdaNode.cpp index 12c6c7992ff..bd0d9f117dc 100644 --- a/src/Analyzer/LambdaNode.cpp +++ b/src/Analyzer/LambdaNode.cpp @@ -49,6 +49,12 @@ String LambdaNode::getName() const return "lambda(" + children[arguments_child_index]->getName() + ") -> " + children[expression_child_index]->getName(); } +bool LambdaNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + return argument_names == rhs_typed.argument_names; +} + void LambdaNode::updateTreeHashImpl(HashState & state) const { state.update(argument_names.size()); diff --git a/src/Analyzer/LambdaNode.h b/src/Analyzer/LambdaNode.h index 3d07ae0583e..a49bf780e12 100644 --- a/src/Analyzer/LambdaNode.h +++ b/src/Analyzer/LambdaNode.h @@ -98,6 +98,8 @@ public: } protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + void updateTreeHashImpl(HashState & state) const override; ASTPtr toASTImpl() const override; diff --git a/src/Analyzer/ListNode.cpp b/src/Analyzer/ListNode.cpp index b8c6a974101..e7f14c62ac2 100644 --- a/src/Analyzer/ListNode.cpp +++ b/src/Analyzer/ListNode.cpp @@ -46,9 +46,15 @@ String ListNode::getName() const return result; } +bool ListNode::isEqualImpl(const IQueryTreeNode &) const +{ + /// No state + return true; +} + void ListNode::updateTreeHashImpl(HashState &) const { - /// Hash automatically updated with children size in IQueryTreeNode.h + /// No state } ASTPtr ListNode::toASTImpl() const diff --git a/src/Analyzer/ListNode.h b/src/Analyzer/ListNode.h index 9071c7c275f..029023b9d8a 100644 --- a/src/Analyzer/ListNode.h +++ b/src/Analyzer/ListNode.h @@ -38,6 +38,8 @@ public: String getName() const override; protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + void updateTreeHashImpl(HashState &) const override; ASTPtr toASTImpl() const override; diff --git a/src/Analyzer/MatcherNode.cpp b/src/Analyzer/MatcherNode.cpp index 4ad38ea6068..af5b0de4990 100644 --- a/src/Analyzer/MatcherNode.cpp +++ b/src/Analyzer/MatcherNode.cpp @@ -182,6 +182,27 @@ String MatcherNode::getName() const return buffer.str(); } +bool MatcherNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + if (matcher_type != rhs_typed.matcher_type || + qualified_identifier != rhs_typed.qualified_identifier || + columns_identifiers != rhs_typed.columns_identifiers || + columns_identifiers_set != rhs_typed.columns_identifiers_set) + return false; + + const auto & rhs_columns_matcher = rhs_typed.columns_matcher; + + if (!columns_matcher && !rhs_columns_matcher) + return true; + else if (columns_matcher && !rhs_columns_matcher) + return false; + else if (!columns_matcher && rhs_columns_matcher) + return false; + + return columns_matcher->pattern() == rhs_columns_matcher->pattern(); +} + void MatcherNode::updateTreeHashImpl(HashState & hash_state) const { hash_state.update(static_cast(matcher_type)); diff --git a/src/Analyzer/MatcherNode.h b/src/Analyzer/MatcherNode.h index 924c028f8f9..1d9d77d9b36 100644 --- a/src/Analyzer/MatcherNode.h +++ b/src/Analyzer/MatcherNode.h @@ -146,6 +146,8 @@ public: String getName() const override; protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + void updateTreeHashImpl(HashState & hash_state) const override; ASTPtr toASTImpl() const override; diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index f4f1856e37f..7ec7ad21659 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -1894,8 +1894,13 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes { String node_alias = node->getAlias(); - /// Do not update node from alias table if we resolve it for duplicate alias - if (!node_alias.empty() && scope.nodes_with_duplicated_aliases.contains(node)) + /** Do not use alias table if node has alias same as some other node. + * Example: WITH x -> x + 1 AS lambda SELECT 1 AS lambda; + * During 1 AS lambda resolve if we use alias table we replace node with x -> x + 1 AS lambda. + */ + bool use_alias_table = !scope.nodes_with_duplicated_aliases.contains(node); + + if (!node_alias.empty() && use_alias_table) { /** Node could be potentially resolved by resolving other nodes. * SELECT b, a as b FROM test_table; @@ -1999,7 +2004,7 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes /** Update aliases after expression node was resolved. * Do not update node in alias table if we resolve it for duplicate alias. */ - if (!node_alias.empty() && scope.nodes_with_duplicated_aliases.contains(node)) + if (!node_alias.empty() && use_alias_table) { auto it = scope.alias_name_to_expression_node.find(node_alias); if (it != scope.alias_name_to_expression_node.end()) @@ -2119,13 +2124,12 @@ void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_tree_node, IdentifierR * After scope nodes are resolved, we can compare node with duplicate alias with * node from scope alias table. */ - for (const auto & node : scope.nodes_with_duplicated_aliases) + for (const auto & node_with_duplicated_alias : scope.nodes_with_duplicated_aliases) { - auto node_copy = node; - auto node_alias = node_copy->getAlias(); - resolveExpressionNode(node_copy, scope, true /*allow_lambda_expression*/); + auto node = node_with_duplicated_alias; + auto node_alias = node->getAlias(); + resolveExpressionNode(node, scope, true /*allow_lambda_expression*/); - auto node_tree_hash = node->getTreeHash(); bool has_node_in_alias_table = false; auto it = scope.alias_name_to_expression_node.find(node_alias); @@ -2133,11 +2137,11 @@ void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_tree_node, IdentifierR { has_node_in_alias_table = true; - if (it->second->getTreeHash() != node_tree_hash) + if (!it->second->isEqual(*node)) throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, "Multiple expressions {} and {} for alias {}. In scope {}", - node_copy->formatASTForErrorMessage(), - it->second->formatASTForErrorMessage(), + node->dumpTree(), + it->second->dumpTree(), node_alias, scope.scope_node->formatASTForErrorMessage()); } @@ -2147,10 +2151,10 @@ void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_tree_node, IdentifierR { has_node_in_alias_table = true; - if (it->second->getTreeHash() != node_tree_hash) + if (!it->second->isEqual(*node)) throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, "Multiple expressions {} and {} for alias {}. In scope {}", - node_copy->formatASTForErrorMessage(), + node->formatASTForErrorMessage(), it->second->formatASTForErrorMessage(), node_alias, scope.scope_node->formatASTForErrorMessage()); @@ -2159,7 +2163,7 @@ void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_tree_node, IdentifierR if (!has_node_in_alias_table) throw Exception(ErrorCodes::LOGICAL_ERROR, "Node {} with duplicate alias {} does not exists in alias table. In scope {}", - node_copy->formatASTForErrorMessage(), + node->formatASTForErrorMessage(), node_alias, scope.scope_node->formatASTForErrorMessage()); } diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index e55bd4fc178..c2ceb87ffe3 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -56,9 +56,15 @@ void QueryNode::dumpTree(WriteBuffer & buffer, size_t indent) const } } +bool QueryNode::isEqualImpl(const IQueryTreeNode &) const +{ + /// No state + return true; +} + void QueryNode::updateTreeHashImpl(HashState &) const { - /// TODO: No state + /// No state } ASTPtr QueryNode::toASTImpl() const diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 1ed59b9c5c8..96f60fab0e7 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -96,6 +96,8 @@ public: void dumpTree(WriteBuffer & buffer, size_t indent) const override; protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + void updateTreeHashImpl(HashState &) const override; ASTPtr toASTImpl() const override; diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index 0b7e4f83015..647ef9eaf2e 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -28,6 +28,12 @@ void TableNode::dumpTree(WriteBuffer & buffer, size_t indent) const buffer << ' ' << storage_id.getFullNameNotQuoted(); } +bool TableNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + return storage_id == rhs_typed.storage_id; +} + void TableNode::updateTreeHashImpl(HashState & state) const { auto full_name = storage_id.getFullNameNotQuoted(); diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index d1212dff36c..a195d5c2df4 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -71,6 +71,8 @@ public: String getName() const override; protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + void updateTreeHashImpl(HashState & state) const override; ASTPtr toASTImpl() const override; diff --git a/src/Analyzer/tests/gtest_query_tree_node.cpp b/src/Analyzer/tests/gtest_query_tree_node.cpp index 95317273fd3..56483614bcd 100644 --- a/src/Analyzer/tests/gtest_query_tree_node.cpp +++ b/src/Analyzer/tests/gtest_query_tree_node.cpp @@ -11,9 +11,9 @@ using namespace DB; class SourceNode final : public IQueryTreeNode { public: - void updateTreeHashImpl(HashState & hash_state) const override + QueryTreeNodeType getNodeType() const override { - (void)(hash_state); + return QueryTreeNodeType::TABLE; } void dumpTree(WriteBuffer & buffer, size_t indent) const override @@ -22,9 +22,14 @@ public: (void)(indent); } - QueryTreeNodeType getNodeType() const override + bool isEqualImpl(const IQueryTreeNode &) const override { - return QueryTreeNodeType::TABLE; + return true; + } + + void updateTreeHashImpl(HashState & hash_state) const override + { + (void)(hash_state); } ASTPtr toASTImpl() const override diff --git a/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.sql b/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.sql index 057be94928f..2569dc8ba9c 100644 --- a/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.sql +++ b/tests/queries/0_stateless/02344_analyzer_multiple_aliases_for_expression.sql @@ -17,6 +17,7 @@ SELECT (SELECT 1) AS subquery, (SELECT 1) AS subquery; SELECT 1 AS value, 2 AS value; -- { serverError 179 } SELECT plus(1, 1) AS value, 2 AS value; -- { serverError 179 } SELECT (SELECT 1) AS subquery, 1 AS subquery; -- { serverError 179 } +WITH x -> x + 1 AS lambda, x -> x + 2 AS lambda SELECT lambda(1); -- { serverError 179 } WITH x -> x + 1 AS lambda SELECT (SELECT 1) AS lambda; -- { serverError 179 } WITH x -> x + 1 AS lambda SELECT 1 AS lambda; -- { serverError 179 } SELECT id AS value, value AS value FROM test_table; -- { serverError 179 } From dd28b5886406254c4d1fa94162c552070b30c1a7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 18 Jul 2022 19:20:28 +0200 Subject: [PATCH 056/188] Added support for subqueries --- src/Analyzer/IQueryTreeNode.h | 4 +- src/Analyzer/Identifier.h | 22 +- src/Analyzer/QueryAnalysisPass.cpp | 619 +++++++++++++----- src/Analyzer/QueryNode.cpp | 88 ++- src/Analyzer/QueryNode.h | 37 +- src/Analyzer/QueryTreeBuilder.cpp | 99 +-- .../InterpreterSelectQueryAnalyzer.cpp | 114 ++-- .../InterpreterSelectQueryAnalyzer.h | 15 +- .../02345_analyzer_subqueries.reference | 27 + .../0_stateless/02345_analyzer_subqueries.sql | 51 ++ 10 files changed, 783 insertions(+), 293 deletions(-) create mode 100644 tests/queries/0_stateless/02345_analyzer_subqueries.reference create mode 100644 tests/queries/0_stateless/02345_analyzer_subqueries.sql diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index a126413bfab..1df296ab3de 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -68,7 +68,7 @@ public: */ virtual String getName() const { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method is not supported for {} query node", getNodeTypeName()); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method getName is not supported for {} query node", getNodeTypeName()); } /** Get result type of query tree node that can be used as part of expression. @@ -77,7 +77,7 @@ public: */ virtual DataTypePtr getResultType() const { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method is not supported for {} query node", getNodeTypeName()); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method getResultType is not supported for {} query node", getNodeTypeName()); } /// Dump query tree to string diff --git a/src/Analyzer/Identifier.h b/src/Analyzer/Identifier.h index 654c7021cf5..9d9404f6831 100644 --- a/src/Analyzer/Identifier.h +++ b/src/Analyzer/Identifier.h @@ -21,18 +21,32 @@ public: /// Create Identifier from parts explicit Identifier(const std::vector & parts_) - : full_name(boost::algorithm::join(parts_, ".")) - , parts(parts_) + : parts(parts_) + , full_name(boost::algorithm::join(parts_, ".")) { } - /// Create Identifier from full_name. Full_name is splitted with '.' as separator. + /// Create Identifier from parts + explicit Identifier(std::vector && parts_) + : parts(std::move(parts_)) + , full_name(boost::algorithm::join(parts_, ".")) + { + } + + /// Create Identifier from full name, full name is splitted with '.' as separator. explicit Identifier(const std::string & full_name_) : full_name(full_name_) { boost::split(parts, full_name, [](char c) { return c == '.'; }); } + /// Create Identifier from full name, full name is splitted with '.' as separator. + explicit Identifier(std::string && full_name_) + : full_name(std::move(full_name_)) + { + boost::split(parts, full_name, [](char c) { return c == '.'; }); + } + const std::string & getFullName() const { return full_name; @@ -151,8 +165,8 @@ public: } private: - std::string full_name; std::vector parts; + std::string full_name; }; inline bool operator==(const Identifier & lhs, const Identifier & rhs) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 7ec7ad21659..3b4986b2ff8 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -68,6 +68,8 @@ namespace ErrorCodes extern const int INCORRECT_RESULT_OF_SCALAR_SUBQUERY; extern const int BAD_ARGUMENTS; extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS; + extern const int UNKNOWN_DATABASE; + extern const int UNKNOWN_TABLE; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -118,7 +120,8 @@ namespace ErrorCodes * 3. Try to resolve identifier from tables if scope is query. * Steps 2 and 3 can be changed using prefer_column_name_to_alias setting. * If identifier could not be resolved in current scope, resolution must be continued in parent scopes. - * 4. Try to resolve identifier from parent scopes. + * 4. If it is table lookup, try to resolve identifier from CTE. + * 5. Try to resolve identifier from parent scopes. * * Additional rules about aliases and scopes. * 1. Parent scope cannot refer alias from child scope. @@ -152,7 +155,6 @@ namespace ErrorCodes * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. - * TODO: CTE * TODO: JOIN, ARRAY JOIN * TODO: bulding sets * TODO: Special functions grouping, in. @@ -233,15 +235,7 @@ struct IdentifierLookupHash } }; -struct StorageIDHash -{ - size_t operator()(const StorageID & storage_id) const - { - return std::hash()(storage_id.getFullNameNotQuoted()); - } -}; - -struct StorageColumns +struct TableExpressionColumns { NamesAndTypesList column_names_and_types; std::unordered_set column_identifier_first_parts; @@ -359,6 +353,9 @@ struct IdentifierResolveScope /// Alias name to table expression node std::unordered_map alias_name_to_table_expression_node; + /// CTE name to query node + std::unordered_map cte_name_to_query_node; + /// Nodes with duplicated identifiers std::unordered_set nodes_with_duplicated_aliases; @@ -377,7 +374,7 @@ struct IdentifierResolveScope buffer << "Scope node " << scope_node->formatASTForErrorMessage() << '\n'; buffer << "Identifier lookup to node " << identifier_lookup_to_node.size() << '\n'; for (const auto & [identifier, node] : identifier_lookup_to_node) - buffer << "Identifier " << identifier.dump() << " node " << node->formatASTForErrorMessage() << '\n'; + buffer << "Identifier " << identifier.dump() << " node type " << node->getNodeTypeName() << " format " << node->formatASTForErrorMessage() << '\n'; buffer << "Expression argument name to node " << expression_argument_name_to_node.size() << '\n'; for (const auto & [alias_name, node] : expression_argument_name_to_node) @@ -393,7 +390,11 @@ struct IdentifierResolveScope buffer << "Alias name to table expression node " << alias_name_to_table_expression_node.size() << '\n'; for (const auto & [alias_name, node] : alias_name_to_table_expression_node) - buffer << "Alias name " << alias_name << " table node " << node->formatASTForErrorMessage() << '\n'; + buffer << "Alias name " << alias_name << " node " << node->formatASTForErrorMessage() << '\n'; + + buffer << "CTE name to query node " << cte_name_to_query_node.size() << '\n'; + for (const auto & [cte_name, node] : cte_name_to_query_node) + buffer << "CTE name " << cte_name << " node " << node->formatASTForErrorMessage() << '\n'; buffer << "Nodes with duplicated aliases " << nodes_with_duplicated_aliases.size() << '\n'; for (const auto & node : nodes_with_duplicated_aliases) @@ -450,7 +451,9 @@ private: static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path); - static QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); + static NamesAndTypesList getQueryNodeColumns(const QueryTreeNodePtr & query_node); + + QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); void evaluateScalarSubquery(QueryTreeNodePtr & query_tree_node); @@ -458,15 +461,15 @@ private: QueryTreeNodePtr tryResolveIdentifierFromExpressionArguments(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); - QueryTreeNodePtr tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); + QueryTreeNodePtr tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, bool allow_to_check_parent_scopes = true); QueryTreeNodePtr tryResolveIdentifierFromTables(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); - QueryTreeNodePtr tryResolveIdentifierInCurrentScope(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & current_scope); + // QueryTreeNodePtr tryResolveIdentifierInCurrentScope(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & current_scope); QueryTreeNodePtr tryResolveIdentifierInParentScopes(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); - QueryTreeNodePtr tryResolveIdentifier(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); + QueryTreeNodePtr tryResolveIdentifier(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, bool allow_to_check_parent_scopes = true); /// Resolve query tree nodes functions @@ -480,7 +483,11 @@ private: void resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression); - void resolveQuery(QueryTreeNodePtr & query_tree_node, IdentifierResolveScope & scope); + void initializeQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope); + + void resolveQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope); + + void resolveQuery(QueryTreeNodePtr & query_node, IdentifierResolveScope & scope); /// Query analyzer context ContextPtr context; @@ -488,8 +495,11 @@ private: /// Lambdas that are currently in resolve process std::unordered_set lambdas_in_resolve_process; - /// Storage id to storage columns cache - std::unordered_map storage_id_to_columns_cache; + /// Query nodes where FROM section is in resolve process + std::unordered_set query_node_from_section_in_resolve_process; + + /// Table expression node to columns cache + std::unordered_map table_expression_node_to_columns; }; /// Utility functions implementation @@ -536,10 +546,37 @@ QueryTreeNodePtr QueryAnalyzer::wrapExpressionNodeInTupleElement(QueryTreeNodePt return expression_node; } +/** Get query node columns using projection section + */ +NamesAndTypesList QueryAnalyzer::getQueryNodeColumns(const QueryTreeNodePtr & query_node) +{ + const auto & query_node_typed = query_node->as(); + + NamesAndTypes query_columns; + + const auto & projection_nodes = query_node_typed.getProjection(); + query_columns.reserve(projection_nodes.getNodes().size()); + + for (const auto & projection_node : projection_nodes.getNodes()) + { + auto column_type = projection_node->getResultType(); + std::string column_name; + + if (projection_node->hasAlias()) + column_name = projection_node->getAlias(); + else + column_name = projection_node->getName(); + + query_columns.emplace_back(column_name, column_type); + } + + return {query_columns.begin(), query_columns.end()}; +} + /** Try to get lambda node from sql user defined functions if sql user defined function with function name exists. * Returns lambda node if function exists, nullptr otherwise. */ -QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name, const ContextPtr & context) +QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name) { auto user_defined_function = UserDefinedSQLFunctionFactory::instance().tryGet(function_name); if (!user_defined_function) @@ -751,6 +788,9 @@ public: } else if (auto * query_tree_node = child->as()) { + if (query_tree_node->isCTE()) + return false; + updateAliasesIfNeeded(data, child, false); return false; } @@ -839,9 +879,10 @@ using ScopeAliasVisitor = ScopeAliasVisitorMatcher::Visitor; * * 8. If identifier lookup is in expression context, clone result expression. */ -QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, bool allow_to_check_parent_scopes) { const auto & identifier_bind_part = identifier_lookup.identifier.front(); + auto get_alias_name_to_node_map = [&]() -> std::unordered_map & { if (identifier_lookup.isExpressionLookup()) @@ -886,7 +927,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier /// Resolve expression if necessary if (auto * alias_identifier = it->second->as()) { - it->second = tryResolveIdentifier(IdentifierLookup{alias_identifier->getIdentifier(), identifier_lookup.lookup_context}, scope); + it->second = tryResolveIdentifier(IdentifierLookup{alias_identifier->getIdentifier(), identifier_lookup.lookup_context}, scope, allow_to_check_parent_scopes); if (!it->second) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, @@ -935,19 +976,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier result = tuple_element_result; } - else if (identifier_lookup.isExpressionLookup() && result) - { - /** If expression node was resolved throught aliases we must clone it to keep query tree state valid. - * - * Example: - * If we have query SELECT 1 as a, a - * If we do not clone node, after query analysis query will look like SELECT 1 as a, 1 as a. - * - * This query is broken because multiple aliases with same name are not allowed in query tree. - */ - result = result->clone(); - result->removeAlias(); - } return result; } @@ -972,16 +1000,43 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL return {}; auto from_node = query_scope_node->getFrom(); - auto * table_node = query_scope_node->getFrom()->as(); - if (!table_node) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "From does not contain table node"); + auto * table_node = from_node->as(); + auto * query_node = from_node->as(); + + /** Edge case scenario when subquery in FROM node try to resolve identifier from parent scopes, when FROM is not resolved. + * SELECT subquery.b AS value FROM (SELECT value, 1 AS b) AS subquery; + * TODO: This can be supported + */ + if (query_node_from_section_in_resolve_process.contains(query_scope_node)) + return {}; + + if (!table_node && !query_node) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "FROM does not contain table or query node. Actual {}", from_node->formatASTForErrorMessage()); const auto & identifier = identifier_lookup.identifier; const auto & path_start = identifier.getParts().front(); - auto storage_id = table_node->getStorageID(); - auto & table_name = storage_id.table_name; - auto & database_name = storage_id.database_name; + + std::string table_expression_name; + + /// Valid only for table node + std::string table_name; + std::string database_name; + + if (table_node) + { + const auto & table_storage_id = table_node->getStorageID(); + table_name = table_storage_id.table_name; + database_name = table_storage_id.database_name; + table_expression_name = table_storage_id.getFullNameNotQuoted(); + } + else if (query_node) + { + table_name = query_node->getCTEName(); + + if (query_node->hasAlias()) + table_expression_name = query_node->getAlias(); + } if (identifier_lookup.isTableLookup()) { @@ -1000,13 +1055,15 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL return {}; } - /// TODO: Check if this cache is safe to use in case of different settings - - auto storage_columns_it = storage_id_to_columns_cache.find(table_node->getStorageID()); - if (storage_columns_it == storage_id_to_columns_cache.end()) + auto storage_columns_it = table_expression_node_to_columns.find(from_node.get()); + if (storage_columns_it == table_expression_node_to_columns.end()) { - StorageColumns storage_columns; - storage_columns.column_names_and_types = table_node->getStorageSnapshot()->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); + TableExpressionColumns storage_columns; + + if (table_node) + storage_columns.column_names_and_types = table_node->getStorageSnapshot()->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); + else if (query_node) + storage_columns.column_names_and_types = getQueryNodeColumns(from_node); for (auto & name_and_type_pair : storage_columns.column_names_and_types) { @@ -1014,56 +1071,75 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL storage_columns.column_identifier_first_parts.insert(column_name_identifier.at(0)); } - auto [inserted_it, _] = storage_id_to_columns_cache.emplace(storage_id, std::move(storage_columns)); + auto [inserted_it, _] = table_expression_node_to_columns.emplace(from_node.get(), std::move(storage_columns)); storage_columns_it = inserted_it; } auto & storage_columns = storage_columns_it->second; - /** If identifier first part binds to some column start. Then we can try to find whole identifier in table. - * 1. Try to bind identifier first part to column in table, if true get full identifier from table or throw exception. - * 2. Try to bind identifier first part to table name or table alias, if true remove first part and try to get full identifier from table or throw exception. - * 3. Try to bind identifier first parts to database name and table name, if true remove first two parts and try to get full identifier from table or throw exception. - */ - auto resolve_identifier_from_table_or_throw = [&](size_t drop_first_parts_size) + auto resolve_identifier_from_storage_or_throw = [&](size_t drop_first_parts_size) -> QueryTreeNodePtr { auto identifier_view = IdentifierView(identifier); identifier_view.popFirst(drop_first_parts_size); - auto result_column = storage_columns.column_names_and_types.tryGetByName(std::string(identifier_view.getFullName())); + /** Compound identifier cannot be resolved directly from storage if storage is not table. + * + * Example: SELECT test_table.id.value1.value2 FROM test_table; + * In table storage column test_table.id.value1.value2 will exists. + * + * Example: SELECT test_subquery.compound_expression.value FROM (SELECT compound_expression AS value) AS test_subquery; + * Here there is no column with name test_subquery.compound_expression.value, and additional wrap in tuple element is required. + */ + + std::optional result_column; + bool compound_identifier = identifier_view.getPartsSize() > 1; + + if (!table_node && compound_identifier) + result_column = storage_columns.column_names_and_types.tryGetByName(std::string(identifier_view.at(0))); + else + result_column = storage_columns.column_names_and_types.tryGetByName(std::string(identifier_view.getFullName())); + if (!result_column) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Identifier {} cannot be resolved from table {}. In scope {}", + "Identifier {} cannot be resolved from {}{}. In scope {}", identifier.getFullName(), - storage_id.getFullTableName(), + query_node ? "subquery" : "table", + table_expression_name.empty() ? "" : " with name " + table_expression_name, scope.scope_node->formatASTForErrorMessage()); - return result_column; + QueryTreeNodePtr result_expression = std::make_shared(*result_column, from_node); + + if (!table_node && compound_identifier) + { + IdentifierView nested_path(identifier_view); + nested_path.popFirst(); + auto tuple_element_result = wrapExpressionNodeInTupleElement(result_expression, identifier_view); + resolveFunction(tuple_element_result, scope); + } + + return result_expression; }; + /** If identifier first part binds to some column start. Then we can try to find whole identifier in table. + * 1. Try to bind identifier first part to column in table, if true get full identifier from table or throw exception. + * 2. Try to bind identifier first part to table name or storage alias, if true remove first part and try to get full identifier from table or throw exception. + * Storage alias works for subquery, table function as well. + * 3. Try to bind identifier first parts to database name and table name, if true remove first two parts and try to get full identifier from table or throw exception. + */ if (storage_columns.canBindIdentifier(IdentifierView(identifier))) - { - auto result_column = resolve_identifier_from_table_or_throw(0 /*drop_first_parts_size*/); - return std::make_shared(*result_column, from_node); - } + return resolve_identifier_from_storage_or_throw(0 /*drop_first_parts_size*/); if (identifier.getPartsSize() == 1) return {}; - if (path_start == table_name || (table_node->hasAlias() && path_start == table_node->getAlias())) - { - auto result_column = resolve_identifier_from_table_or_throw(1 /*drop_first_parts_size*/); - return std::make_shared(*result_column, from_node); - } + if ((!table_name.empty() && path_start == table_name) || (from_node->hasAlias() && path_start == from_node->getAlias())) + return resolve_identifier_from_storage_or_throw(1 /*drop_first_parts_size*/); if (identifier.getPartsSize() == 2) return {}; - if (path_start == database_name && identifier[1] == table_name) - { - auto result_column = resolve_identifier_from_table_or_throw(2 /*drop_first_parts_size*/); - return std::make_shared(*result_column, from_node); - } + if (!database_name.empty() && path_start == database_name && identifier[1] == table_name) + return resolve_identifier_from_storage_or_throw(2 /*drop_first_parts_size*/); return {}; } @@ -1077,35 +1153,35 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL * 2. Try to resolve identifier from aliases. * 3. Try to resolve identifier from tables. */ -QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierInCurrentScope(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & current_scope) -{ - auto resolved_identifier = tryResolveIdentifierFromExpressionArguments(identifier_lookup, current_scope); - if (resolved_identifier) - return resolved_identifier; +// QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierInCurrentScope(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & current_scope) +// { +// auto resolved_identifier = tryResolveIdentifierFromExpressionArguments(identifier_lookup, current_scope); +// if (resolved_identifier) +// return resolved_identifier; - bool prefer_column_name_to_alias = context->getSettingsRef().prefer_column_name_to_alias; - if (unlikely(prefer_column_name_to_alias)) - { - resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, current_scope); - if (resolved_identifier) - return resolved_identifier; +// bool prefer_column_name_to_alias = context->getSettingsRef().prefer_column_name_to_alias; +// if (unlikely(prefer_column_name_to_alias)) +// { +// resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, current_scope); +// if (resolved_identifier) +// return resolved_identifier; - return tryResolveIdentifierFromAliases(identifier_lookup, current_scope); - } +// return tryResolveIdentifierFromAliases(identifier_lookup, current_scope); +// } - resolved_identifier = tryResolveIdentifierFromAliases(identifier_lookup, current_scope); - if (resolved_identifier) - return resolved_identifier; +// resolved_identifier = tryResolveIdentifierFromAliases(identifier_lookup, current_scope); +// if (resolved_identifier) +// return resolved_identifier; - return tryResolveIdentifierFromTables(identifier_lookup, current_scope); -} +// return tryResolveIdentifierFromTables(identifier_lookup, current_scope); +// } /** Try resolve identifier in current scope parent scopes. * If initial scope is query. Then return nullptr. * TODO: CTE, constants can be used from parent query with statement. * TODO: If column is matched, throw exception that nested subqueries are not supported. * - * If iniital scope is expression. Then try to resolve identifier in parent scopes until query scope is hit. + * If initial scope is expression. Then try to resolve identifier in parent scopes until query scope is hit. * For query scope resolve strategy is same as if initial scope if query. */ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierInParentScopes(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) @@ -1116,39 +1192,71 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierInParentScopes(const Identif bool initial_scope_is_query = scope.scope_node->getNodeType() == QueryTreeNodeType::QUERY; bool initial_scope_is_expression = !initial_scope_is_query; + IdentifierResolveScope * scope_to_check = scope.parent_scope; + if (initial_scope_is_expression) { - IdentifierResolveScope * scope_to_check = scope.parent_scope; while (scope_to_check != nullptr) { - auto resolved_identifier = tryResolveIdentifier(identifier_lookup, *scope_to_check); + auto resolved_identifier = tryResolveIdentifier(identifier_lookup, *scope_to_check, false /*allow_to_check_parent_scopes*/); if (resolved_identifier) return resolved_identifier; - if (scope_to_check->scope_node->getNodeType() == QueryTreeNodeType::QUERY) - break; - + bool scope_was_query = scope_to_check->scope_node->getNodeType() == QueryTreeNodeType::QUERY; scope_to_check = scope_to_check->parent_scope; + + if (scope_was_query) + break; } } + while (scope_to_check != nullptr) + { + auto resolved_identifier = tryResolveIdentifier(identifier_lookup, *scope_to_check, false /*allow_to_check_parent_scopes*/); + if (resolved_identifier) + { + if (resolved_identifier->as()) + return resolved_identifier; + else if (resolved_identifier->as() && resolved_identifier->as()->isCTE()) + return resolved_identifier; + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Resolve identifier {} from parent scope only supported for constants and CTE. In scope {}", + identifier_lookup.identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + } + + scope_to_check = scope_to_check->parent_scope; + } + return {}; } /** Resolve identifier in scope. + * * Steps: * 1. Register identifier lookup in scope identifier_lookup_to_resolve_status table. * If entry is already registered and is not resolved, that means that we have cyclic aliases for identifier. * Example: SELECT a AS b, b AS a; - * 2. Try resolve identifier in current scope. - * 3. If identifier is not resolved in current scope, try to resolve it in parent scopes. - * 4. If identifier was not resolved remove it from identifier_lookup_to_resolve_status table. + * Try resolve identifier in current scope: + * 3. Try resolve identifier from expression arguments. + * If prefer_column_name_to_alias = true. + * + * 4. Try to resolve identifier from tables. + * 5. Try to resolve identifier from aliases. + * Otherwise. + * 4. Try to resolve identifier from aliases. + * 5. Try to resolve identifier from tables. + * + * 6. If identifier is not resolved in current scope, try to resolve it in parent scopes. + * 7. If identifier was not resolved remove it from identifier_lookup_to_resolve_status table. + * * It is okay for identifier to be not resolved, in case we want first try to lookup identifier in one context, * then if there is no identifier in this context, try to lookup in another context. * Example: Try to lookup identifier as function, if it is not found lookup as expression. * Example: Try to lookup identifier as expression, if it is not found lookup as table. */ -QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifier(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifier(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, bool allow_to_check_parent_scopes) { auto it = scope.identifier_lookup_to_node.find(identifier_lookup); if (it != scope.identifier_lookup_to_node.end()) @@ -1159,21 +1267,51 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifier(const IdentifierLookup & id identifier_lookup.identifier.getFullName(), scope.scope_node->formatASTForErrorMessage()); - if (scope.non_cached_identifier_lookups_during_expression_resolve.count(identifier_lookup) == 0) + if (!scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup)) return it->second; } auto [insert_it, _] = scope.identifier_lookup_to_node.insert({identifier_lookup, QueryTreeNodePtr()}); it = insert_it; - QueryTreeNodePtr resolved_identifier = tryResolveIdentifierInCurrentScope(identifier_lookup, scope); + /// Resolve identifier from current scope + + auto resolved_identifier = tryResolveIdentifierFromExpressionArguments(identifier_lookup, scope); if (!resolved_identifier) + { + bool prefer_column_name_to_alias = context->getSettingsRef().prefer_column_name_to_alias; + + if (unlikely(prefer_column_name_to_alias)) + { + resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, scope); + if (!resolved_identifier) + resolved_identifier = tryResolveIdentifierFromAliases(identifier_lookup, scope, allow_to_check_parent_scopes); + } + else + { + resolved_identifier = tryResolveIdentifierFromAliases(identifier_lookup, scope, allow_to_check_parent_scopes); + + if (!resolved_identifier) + resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, scope); + } + } + + if (!resolved_identifier && identifier_lookup.isTableLookup()) + { + auto cte_query_node_it = scope.cte_name_to_query_node.find(identifier_lookup.identifier.getFullName()); + if (cte_query_node_it != scope.cte_name_to_query_node.end()) + resolved_identifier = cte_query_node_it->second; + } + + /// Try to resolve identifier from parent scopes + + if (!resolved_identifier && allow_to_check_parent_scopes) resolved_identifier = tryResolveIdentifierInParentScopes(identifier_lookup, scope); it->second = resolved_identifier; - if (!resolved_identifier || scope.non_cached_identifier_lookups_during_expression_resolve.count(identifier_lookup) > 0) + if (!resolved_identifier || scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup)) scope.identifier_lookup_to_node.erase(it); return resolved_identifier; @@ -1237,22 +1375,33 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, } else { - auto table_query_tree_node = tryResolveIdentifier({matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::TABLE}, scope); + auto table_expression_node = tryResolveIdentifier({matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::TABLE}, scope, false /*allow_to_check_parent_scopes*/); - if (!table_query_tree_node || table_query_tree_node->getNodeType() != QueryTreeNodeType::TABLE) + if (!table_expression_node || (table_expression_node->as() && table_expression_node->as()->isCTE())) + { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Qualified matcher {} does not find table. In scope {}", matcher_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); + } - const auto & table_node = table_query_tree_node->as(); - auto initial_matcher_columns = table_node.getStorageSnapshot()->getColumns(GetColumnsOptions(GetColumnsOptions::All)); + NamesAndTypesList initial_matcher_columns; + + if (auto * table_expression_query_node = table_expression_node->as()) + { + initial_matcher_columns = getQueryNodeColumns(table_expression_node); + } + else + { + const auto & table_expression_table_node = table_expression_node->as(); + initial_matcher_columns = table_expression_table_node.getStorageSnapshot()->getColumns(GetColumnsOptions(GetColumnsOptions::All)); + } for (auto & column : initial_matcher_columns) { const auto & column_name = column.name; if (matcher_node_typed.isMatchingColumn(column_name)) - matcher_expression_nodes.push_back(std::make_shared(column, table_query_tree_node)); + matcher_expression_nodes.push_back(std::make_shared(column, table_expression_node)); } } } @@ -1282,28 +1431,37 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, scope.scope_node->formatASTForErrorMessage()); } - const auto & table_node = scope_query_node->getFrom()->as(); + NamesAndTypesList initial_matcher_columns; - UInt8 get_column_options_kind = 0; - - if (matcher_node_typed.isAsteriskMatcher()) + if (auto * from_query_node = scope_query_node->getFrom()->as()) { - get_column_options_kind = GetColumnsOptions::Ordinary; - - if (context->getSettingsRef().asterisk_include_alias_columns) - get_column_options_kind |= GetColumnsOptions::Kind::Aliases; - - if (context->getSettingsRef().asterisk_include_materialized_columns) - get_column_options_kind |= GetColumnsOptions::Kind::Materialized; + initial_matcher_columns = getQueryNodeColumns(scope_query_node->getFrom()); } else { - /// TODO: Check if COLUMNS select aliases column by default - get_column_options_kind = GetColumnsOptions::All; - } + const auto & table_node = scope_query_node->getFrom()->as(); - auto get_columns_options = GetColumnsOptions(static_cast(get_column_options_kind)); - auto initial_matcher_columns = table_node.getStorageSnapshot()->getColumns(get_columns_options); + UInt8 get_column_options_kind = 0; + + if (matcher_node_typed.isAsteriskMatcher()) + { + get_column_options_kind = GetColumnsOptions::Ordinary; + + if (context->getSettingsRef().asterisk_include_alias_columns) + get_column_options_kind |= GetColumnsOptions::Kind::Aliases; + + if (context->getSettingsRef().asterisk_include_materialized_columns) + get_column_options_kind |= GetColumnsOptions::Kind::Materialized; + } + else + { + /// TODO: Check if COLUMNS select aliases column by default + get_column_options_kind = GetColumnsOptions::All; + } + + auto get_columns_options = GetColumnsOptions(static_cast(get_column_options_kind)); + initial_matcher_columns = table_node.getStorageSnapshot()->getColumns(get_columns_options); + } for (auto & column : initial_matcher_columns) { @@ -1670,8 +1828,8 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc * If no lambda node exists with function node name identifier, try to resolve it as lambda from sql user defined functions. */ auto lambda_expression_untyped = tryResolveIdentifier({Identifier{function_node.getFunctionName()}, IdentifierLookupContext::FUNCTION}, scope); - // if (!lambda_expression_untyped) - // lambda_expression_untyped = tryGetLambdaFromSQLUserDefinedFunction(function_node.getFunctionName()); + if (!lambda_expression_untyped) + lambda_expression_untyped = tryGetLambdaFromSQLUserDefinedFunction(function_node.getFunctionName()); /** If function is resolved as lambda. * Clone lambda before resolve. @@ -1950,6 +2108,8 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes unresolved_identifier.getFullName(), scope.scope_node->formatASTForErrorMessage()); } + + node->setAlias(node_alias); } else if (auto * function_node = node->as()) { @@ -1973,7 +2133,7 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes /// Must be resolved by caller } - else if (auto * asterisk_node = node->as()) + else if (auto * matcher_node = node->as()) { node = resolveMatcher(node, scope); } @@ -2025,6 +2185,7 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes } /** Resolve expression node list. + * If expression is CTE subquery node it is skipped. * If expression is resolved in list, it is flattened into initial node list. * * Such examples must work: @@ -2041,6 +2202,10 @@ void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, Iden for (auto & node : initial_node_list.getNodes()) { + /// Skip CTE + if (node->as() && node->as()->isCTE()) + continue; + resolveExpressionNode(node, scope, allow_lambda_expression); if (auto * expression_list = node->as()) @@ -2057,6 +2222,118 @@ void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, Iden node_list = std::move(result_node_list); } +/** Initialize query FROM section. + */ +void QueryAnalyzer::initializeQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope) +{ + if (from_node->getNodeType() == QueryTreeNodeType::QUERY || + from_node->getNodeType() == QueryTreeNodeType::TABLE) + { + /// Already initialized + } + else if (auto * from_table_identifier = from_node->as()) + { + /// TODO: Context resolve storage + + const auto & storage_identifier = from_table_identifier->getIdentifier(); + size_t parts_size = storage_identifier.getPartsSize(); + if (parts_size < 1 || parts_size > 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table identifier should consist of 1 or 2 parts. Actual {}", storage_identifier.getFullName()); + + std::string database_name; + std::string table_name; + + if (storage_identifier.isCompound()) + { + database_name = storage_identifier[0]; + table_name = storage_identifier[1]; + } + else + { + table_name = storage_identifier[0]; + } + + bool resolved_as_cte = false; + + if (database_name.empty()) + { + auto table_identifier_lookup = IdentifierLookup{Identifier(table_name), IdentifierLookupContext::TABLE}; + auto resolved_identifier = tryResolveIdentifier(table_identifier_lookup, scope, true /*allow_to_check_parent_scopes*/); + + if (resolved_identifier && resolved_identifier->as() && resolved_identifier->as()->isCTE()) + { + auto from_node_alias = from_node->getAlias(); + from_node = resolved_identifier->clone(); + from_node->as().setIsCTE(false); + from_node->setAlias(from_node_alias); + + resolved_as_cte = true; + } + + database_name = context->getCurrentDatabase(); + } + + if (!resolved_as_cte) + { + auto & database_catalog = DatabaseCatalog::instance(); + auto database = database_catalog.tryGetDatabase(database_name); + if (!database) + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exists", database_name); + + auto table = database->tryGetTable(table_name, context); + + if (!table) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exists", table_name); + + auto from_node_alias = from_node->getAlias(); + from_node = std::make_shared(std::move(table), context); + from_node->setAlias(from_node_alias); + } + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Query FROM section expected table or query. Actual {} {}. In scope {}", + from_node->getNodeTypeName(), + from_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + + auto from_node_alias = from_node->getAlias(); + if (!from_node_alias.empty()) + { + auto [_, inserted] = scope.alias_name_to_table_expression_node.insert(std::make_pair(from_node_alias, from_node)); + if (!inserted) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Duplicate aliases for tables in FROM section are not allowed", + from_node_alias); + } +} + +/** Resolve query FROM section. + * + * Resolve steps: + * 1. If FROM node is query, resolve it. + * 2. If FROM node is table is it already resolved. + */ +void QueryAnalyzer::resolveQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope) +{ + if (from_node->getNodeType() == QueryTreeNodeType::QUERY) + { + IdentifierResolveScope subquery_scope(from_node, &scope); + resolveQuery(from_node, subquery_scope); + return; + } + else if (auto * from_table = from_node->as()) + { + return; + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Query FROM section expected table or query. Actual {}", from_node->formatASTForErrorMessage()); + } +} + /** Resolve query. * This function modifies query node during resolve. It is caller responsibility to clone query node before resolve * if it is needed for later use. @@ -2066,57 +2343,93 @@ void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, Iden * * Resolve steps: * 1. Initialize query scope with aliases. - * 2. Resolve expressions in query parts. - * 3. Remove WITH section from query. + * 2. Register CTE subqueries in scope. + * 3. Resolve FROM section. + * 4. Resolve expressions in query parts. + * 5. Remove WITH section from query. + * 6. Validate nodes with duplicate aliases. */ -void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_tree_node, IdentifierResolveScope & scope) +void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_node, IdentifierResolveScope & scope) { - auto & query_tree = query_tree_node->as(); + auto & query_node_typed = query_node->as(); /// Initialize aliases in query node scope ScopeAliasVisitorMatcher::Data data{scope}; ScopeAliasVisitorMatcher::Visitor visitor(data); - if (query_tree.getWithNode()) - visitor.visit(query_tree.getWithNode()); + if (query_node_typed.getWithNode()) + visitor.visit(query_node_typed.getWithNode()); - if (query_tree.getProjectionNode()) - visitor.visit(query_tree.getProjectionNode()); + if (query_node_typed.getProjectionNode()) + visitor.visit(query_node_typed.getProjectionNode()); - if (query_tree.getPrewhere()) - visitor.visit(query_tree.getPrewhere()); + if (query_node_typed.getPrewhere()) + visitor.visit(query_node_typed.getPrewhere()); - if (query_tree.getWhere()) - visitor.visit(query_tree.getWhere()); + if (query_node_typed.getWhere()) + visitor.visit(query_node_typed.getWhere()); - auto from_node_alias = query_tree.getFrom()->getAlias(); - if (!from_node_alias.empty()) + /// Register CTE subqueries + + if (query_node_typed.getWithNode()) { - auto [_, inserted] = scope.alias_name_to_table_expression_node.insert(std::make_pair(from_node_alias, query_tree.getFrom())); - if (!inserted) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Multiple aliases does not point to same entity {}", from_node_alias); + for (auto & node : query_node_typed.getWith().getNodes()) + { + auto * subquery_node = node->as(); + if (!subquery_node || !subquery_node->isCTE()) + continue; + + const auto & cte_name = subquery_node->getCTEName(); + auto [_, inserted] = data.scope.cte_name_to_query_node.emplace(cte_name, node); + if (!inserted) + throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, + "CTE with name {} already exists. In scope {}", + cte_name, + data.scope.scope_node->formatASTForErrorMessage()); + } } + if (query_node_typed.getFrom()) + { + auto [it, _] = query_node_from_section_in_resolve_process.emplace(query_node.get()); + + initializeQueryFrom(query_node_typed.getFrom(), scope); + resolveQueryFrom(query_node_typed.getFrom(), scope); + + query_node_from_section_in_resolve_process.erase(it); + } + + /** Clear cache of identifier lookups. + * It is necessary because during resolve of FROM section we could not use FROM section. + * + * Example: + * WITH a AS (SELECT 1) FROM SELECT a.* FROM a; + * + * During FROM section resolve identifier a is resolved into CTE. + * During qualified matcher a.* resolve identifier a must be resolved into table. + */ + scope.identifier_lookup_to_node.clear(); + /// Resolve query node sections. - if (query_tree.getWithNode()) - resolveExpressionNodeList(query_tree.getWithNode(), scope, true /*allow_lambda_expression*/); + if (query_node_typed.getWithNode()) + resolveExpressionNodeList(query_node_typed.getWithNode(), scope, true /*allow_lambda_expression*/); - resolveExpressionNodeList(query_tree.getProjectionNode(), scope, false /*allow_lambda_expression*/); + resolveExpressionNodeList(query_node_typed.getProjectionNode(), scope, false /*allow_lambda_expression*/); - if (query_tree.getPrewhere()) - resolveExpressionNode(query_tree.getPrewhere(), scope, false /*allow_lambda_expression*/); + if (query_node_typed.getPrewhere()) + resolveExpressionNode(query_node_typed.getPrewhere(), scope, false /*allow_lambda_expression*/); - if (query_tree.getWhere()) - resolveExpressionNode(query_tree.getWhere(), scope, false /*allow_lambda_expression*/); + if (query_node_typed.getWhere()) + resolveExpressionNode(query_node_typed.getWhere(), scope, false /*allow_lambda_expression*/); /** WITH section can be safely removed, because WITH section only can provide aliases to expressions * and CTE for other sections to use. * * Example: WITH 1 AS constant, (x -> x + 1) AS lambda, a AS (SELECT * FROM test_table); */ - query_tree.getWithNode() = std::make_shared(); + query_node_typed.getWithNode() = std::make_shared(); /** Resolve nodes with duplicate aliases. * diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index c2ceb87ffe3..67a66644eb4 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -1,7 +1,12 @@ #include -#include +#include #include +#include +#include +#include + +#include #include #include @@ -21,50 +26,56 @@ void QueryNode::dumpTree(WriteBuffer & buffer, size_t indent) const { buffer << std::string(indent, ' ') << "QUERY "; writePointerHex(this, buffer); - buffer << '\n'; + + if (hasAlias()) + buffer << " alias : " << getAlias(); + + buffer << " subquery : " << is_subquery; + buffer << " cte : " << is_cte; + buffer << " cte_name : " << cte_name; if (!getWith().getNodes().empty()) { - buffer << std::string(indent, ' ') << "WITH\n"; - getWith().dumpTree(buffer, indent + 2); - buffer << '\n'; + buffer << '\n' << std::string(indent + 2, ' ') << "WITH\n"; + getWith().dumpTree(buffer, indent + 4); } - buffer << std::string(indent, ' ') << "PROJECTION\n"; - getProjection().dumpTree(buffer, indent + 2); buffer << '\n'; + buffer << std::string(indent + 2, ' ') << "PROJECTION\n"; + getProjection().dumpTree(buffer, indent + 4); if (getFrom()) { - buffer << std::string(indent, ' ') << "JOIN TREE\n"; - getFrom()->dumpTree(buffer, indent + 2); - buffer << '\n'; + buffer << '\n' << std::string(indent + 2, ' ') << "JOIN TREE\n"; + getFrom()->dumpTree(buffer, indent + 4); } if (getPrewhere()) { - buffer << std::string(indent, ' ') << "PREWHERE\n"; - getPrewhere()->dumpTree(buffer, indent + 2); - buffer << '\n'; + buffer << '\n' << std::string(indent + 2, ' ') << "PREWHERE\n"; + getPrewhere()->dumpTree(buffer, indent + 4); } if (getWhere()) { - buffer << std::string(indent, ' ') << "WHERE\n"; - getWhere()->dumpTree(buffer, indent + 2); - buffer << '\n'; + buffer << '\n' << std::string(indent + 2, ' ') << "WHERE\n"; + getWhere()->dumpTree(buffer, indent + 4); } } -bool QueryNode::isEqualImpl(const IQueryTreeNode &) const +bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const { - /// No state - return true; + const auto & rhs_typed = assert_cast(rhs); + return is_subquery == rhs_typed.is_subquery && cte_name == rhs_typed.cte_name; } -void QueryNode::updateTreeHashImpl(HashState &) const +void QueryNode::updateTreeHashImpl(HashState & state) const { - /// No state + state.update(is_subquery); + state.update(is_cte); + + state.update(cte_name.size()); + state.update(cte_name); } ASTPtr QueryNode::toASTImpl() const @@ -78,7 +89,11 @@ ASTPtr QueryNode::toASTImpl() const auto table_expression_ast = std::make_shared(); table_expression_ast->children.push_back(children[from_child_index]->toAST()); - table_expression_ast->database_and_table_name = table_expression_ast->children.back(); + + if (children[from_child_index]->getNodeType() == QueryTreeNodeType::TABLE) + table_expression_ast->database_and_table_name = table_expression_ast->children.back(); + else if (children[from_child_index]->getNodeType() == QueryTreeNodeType::QUERY) + table_expression_ast->subquery = table_expression_ast->children.back(); auto tables_in_select_query_element_ast = std::make_shared(); tables_in_select_query_element_ast->children.push_back(std::move(table_expression_ast)); @@ -95,12 +110,37 @@ ASTPtr QueryNode::toASTImpl() const if (getWhere()) select_query->setExpression(ASTSelectQuery::Expression::WHERE, getWhere()->toAST()); - return select_query; + auto result_select_query = std::make_shared(); + result_select_query->union_mode = SelectUnionMode::Unspecified; + + auto list_of_selects = std::make_shared(); + list_of_selects->children.push_back(std::move(select_query)); + + result_select_query->list_of_selects = std::move(list_of_selects); + result_select_query->children.push_back(std::move(select_query)); + + if (is_subquery) + { + auto subquery = std::make_shared(); + + subquery->cte_name = cte_name; + subquery->children.push_back(std::move(result_select_query)); + + return subquery; + } + + return result_select_query; } QueryTreeNodePtr QueryNode::cloneImpl() const { - return std::make_shared(); + auto result_query_node = std::make_shared(); + + result_query_node->is_subquery = is_subquery; + result_query_node->is_cte = is_cte; + result_query_node->cte_name = cte_name; + + return result_query_node; } } diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 96f60fab0e7..b8d0d6a0d88 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -8,7 +8,6 @@ namespace DB { /** Query node represents query in query tree. - * TODO: CTE. */ class QueryNode; using QueryNodePtr = std::shared_ptr; @@ -16,7 +15,37 @@ using QueryNodePtr = std::shared_ptr; class QueryNode final : public IQueryTreeNode { public: - QueryNode(); + explicit QueryNode(); + + bool isSubquery() const + { + return is_subquery; + } + + void setIsSubquery(bool is_subquery_value) + { + is_subquery = is_subquery_value; + } + + bool isCTE() const + { + return is_cte; + } + + void setIsCTE(bool is_cte_value) + { + is_cte = is_cte_value; + } + + const std::string & getCTEName() const + { + return cte_name; + } + + void setCTEName(std::string cte_name_value) + { + cte_name = std::move(cte_name_value); + } const ListNode & getWith() const { @@ -105,6 +134,10 @@ protected: QueryTreeNodePtr cloneImpl() const override; private: + bool is_subquery = false; + bool is_cte = false; + std::string cte_name; + static constexpr size_t with_child_index = 0; static constexpr size_t projection_child_index = 1; static constexpr size_t from_child_index = 2; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index b5d1ad837ed..e134e65f309 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -62,9 +62,9 @@ public: } private: - QueryTreeNodePtr getSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_scalar_query) const; + QueryTreeNodePtr getSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_subquery, const std::string & cte_name) const; - QueryTreeNodePtr getSelectExpression(const ASTPtr & select_query, bool is_scalar_query) const; + QueryTreeNodePtr getSelectExpression(const ASTPtr & select_query, bool is_subquery, const std::string & cte_name) const; QueryTreeNodePtr getExpressionList(const ASTPtr & expression_list) const; @@ -74,8 +74,6 @@ private: ColumnTransformersNodes getColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const; - StoragePtr resolveStorage(const Identifier & storage_identifier) const; - ASTPtr query; QueryTreeNodePtr query_tree_node; @@ -86,23 +84,23 @@ QueryTreeBuilder::QueryTreeBuilder(ASTPtr query_, ContextPtr context_) , query(query_->clone()) { if (query->as()) - query_tree_node = getSelectWithUnionExpression(query, false /*is_scalar_query*/); + query_tree_node = getSelectWithUnionExpression(query, false /*is_subquery*/, {} /*cte_name*/); else if (query->as()) - query_tree_node = getSelectExpression(query, false /*is_scalar_query*/); + query_tree_node = getSelectExpression(query, false /*is_subquery*/, {} /*cte_name*/); else if (query->as()) query_tree_node = getExpressionList(query); else query_tree_node = getExpression(query); } -QueryTreeNodePtr QueryTreeBuilder::getSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_scalar_query) const +QueryTreeNodePtr QueryTreeBuilder::getSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_subquery, const std::string & cte_name) const { auto & select_with_union_query_typed = select_with_union_query->as(); auto & select_lists = select_with_union_query_typed.list_of_selects->as(); if (select_lists.children.size() == 1) { - return getSelectExpression(select_with_union_query->children[0]->children[0], is_scalar_query); + return getSelectExpression(select_with_union_query->children[0]->children[0], is_subquery, cte_name); } else { @@ -125,10 +123,15 @@ QueryTreeNodePtr QueryTreeBuilder::getSelectWithUnionExpression(const ASTPtr & s } } -QueryTreeNodePtr QueryTreeBuilder::getSelectExpression(const ASTPtr & select_query, bool) const +QueryTreeNodePtr QueryTreeBuilder::getSelectExpression(const ASTPtr & select_query, bool is_subquery, const std::string & cte_name) const { const auto & select_query_typed = select_query->as(); auto current_query_tree = std::make_shared(); + + current_query_tree->setIsSubquery(is_subquery); + current_query_tree->setIsCTE(!cte_name.empty()); + current_query_tree->setCTEName(cte_name); + current_query_tree->getFrom() = getFromNode(select_query_typed.tables()); current_query_tree->setOriginalAST(select_query); @@ -281,7 +284,15 @@ QueryTreeNodePtr QueryTreeBuilder::getExpression(const ASTPtr & expression) cons else if (const auto * subquery = expression->as()) { auto subquery_query = subquery->children[0]; - auto query_node = getSelectWithUnionExpression(subquery_query, false); + auto query_node = getSelectWithUnionExpression(subquery_query, true /*is_subquery*/, {} /*cte_name*/); + + result = query_node; + } + else if (const auto * with_element = expression->as()) + { + auto with_element_subquery = with_element->subquery->as().children.at(0); + auto query_node = getSelectWithUnionExpression(with_element_subquery, true /*is_subquery*/, with_element->name /*cte_name*/); + result = query_node; } else if (const auto * columns_regexp_matcher = expression->as()) @@ -344,8 +355,7 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q * SELECT * FROM system.one; */ Identifier storage_identifier("system.one"); - auto table = resolveStorage(storage_identifier); - return std::make_shared(table, getContext()); + return std::make_shared(storage_identifier); } auto & tables = tables_in_select_query->as(); @@ -358,32 +368,31 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q if (table_element.table_expression) { - auto * table_expression = table_element.table_expression->as(); + auto & table_expression = table_element.table_expression->as(); - if (table_expression->database_and_table_name) + if (table_expression.database_and_table_name) { - /// Add CTE support - auto & table_identifier_typed = table_expression->database_and_table_name->as(); + auto & table_identifier_typed = table_expression.database_and_table_name->as(); auto storage_identifier = Identifier(table_identifier_typed.name_parts); - auto table = resolveStorage(storage_identifier); - auto node = std::make_shared(table, getContext()); + auto node = std::make_shared(storage_identifier); node->setAlias(table_identifier_typed.tryGetAlias()); node->setOriginalAST(table_element.table_expression); return node; } - // else if (auto * subquery_expression = table_expression->subquery->as()) - // { - // const auto & select_with_union_query = subquery_expression->children[0]; - // auto expression = getSelectWithUnionExpression(select_with_union_query, false /*scalar query*/); - // expression->setAlias(subquery_expression->tryGetAlias()); + else if (table_expression.subquery) + { + auto & subquery_expression = table_expression.subquery->as(); + const auto & select_with_union_query = subquery_expression.children[0]; - // if (auto * select_expression = expression->as()) - // scope->addInnerScope(select_expression->getScope()); + auto node = getSelectWithUnionExpression(select_with_union_query, true /*is_subquery*/, {} /*cte_name*/); - // table_expressions.emplace_back(std::move(expression)); - // } + node->setAlias(subquery_expression.tryGetAlias()); + node->setOriginalAST(select_with_union_query); + + return node; + } else { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only table is supported"); @@ -505,42 +514,6 @@ ColumnTransformersNodes QueryTreeBuilder::getColumnTransformers(const ASTPtr & m return column_transformers; } -StoragePtr QueryTreeBuilder::resolveStorage(const Identifier & storage_identifier) const -{ - size_t parts_size = storage_identifier.getPartsSize(); - if (parts_size < 1 || parts_size > 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table identifier should consist of 1 or 2 parts"); - - std::string database_name; - std::string table_name; - - if (storage_identifier.isCompound()) - { - database_name = storage_identifier[0]; - table_name = storage_identifier[1]; - } - else - { - table_name = storage_identifier[0]; - } - - auto current_context = getContext(); - if (database_name.empty()) - database_name = current_context->getCurrentDatabase(); - - auto & database_catalog = DatabaseCatalog::instance(); - auto database = database_catalog.tryGetDatabase(database_name); - if (!database) - throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exists", database_name); - - auto table = database->tryGetTable(table_name, current_context); - - if (!table) - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exists", table_name); - - return table; -} - QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context) { QueryTreeBuilder builder(query, context); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 73da6ea827e..11456ead44d 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -439,38 +440,62 @@ public: using CollectSourceColumnsVisitor = CollectSourceColumnsMatcher::Visitor; -InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer(const ASTPtr & query_ptr_, +InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( + const ASTPtr & query_, const SelectQueryOptions & select_query_options_, ContextPtr context_) : WithContext(context_) - , query_ptr(query_ptr_) + , query(query_) , select_query_options(select_query_options_) - , query_tree_pass_manager(context_) { - addQueryTreePasses(query_tree_pass_manager); - - if (auto * select_with_union_query_typed = query_ptr->as()) + if (auto * select_with_union_query_typed = query->as()) { auto & select_lists = select_with_union_query_typed->list_of_selects->as(); if (select_lists.children.size() == 1) { - query_ptr = select_lists.children[0]; + query = select_lists.children[0]; } else { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "UNION is not supported"); } } - else if (auto * select_query_typed = query_ptr->as()) + else if (auto * subquery = query->as()) + { + query = subquery->children[0]; + } + else if (auto * select_query_typed = query_->as()) { } else { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected ASTSelectWithUnionQuery or ASTSelectQuery. Actual {}", - query_ptr_->formatForErrorMessage()); + query->formatForErrorMessage()); } + + query_tree = buildQueryTree(query, context_); + + QueryTreePassManager query_tree_pass_manager(context_); + addQueryTreePasses(query_tree_pass_manager); + query_tree_pass_manager.run(query_tree); +} + +InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( + const QueryTreeNodePtr & query_tree_, + const SelectQueryOptions & select_query_options_, + ContextPtr context_) + : WithContext(context_) + , query(query_tree_->toAST()) + , query_tree(query_tree_) + , select_query_options(select_query_options_) +{ + if (query_tree_->getNodeType() != QueryTreeNodeType::QUERY) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Expected query node. Actual {}", + query_tree_->formatASTForErrorMessage()); + } Block InterpreterSelectQueryAnalyzer::getSampleBlock() @@ -498,28 +523,22 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() if (query_plan.isInitialized()) return; - auto query_tree_untyped = buildQueryTree(query_ptr, getContext()); - auto query_tree = std::static_pointer_cast(query_tree_untyped); - - auto * table_node = query_tree->getFrom()->as(); - if (!table_node) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only single table is supported"); - - query_tree_pass_manager.run(query_tree); + auto & query_tree_typed = query_tree->as(); ActionsDAGPtr action_dag = std::make_shared(); ColumnsWithTypeAndName inputs; CollectSourceColumnsVisitor::Data data; CollectSourceColumnsVisitor collect_source_columns_visitor(data); - collect_source_columns_visitor.visit(query_tree_untyped); + collect_source_columns_visitor.visit(query_tree); + NameSet source_columns_set = std::move(data.source_columns_set); // std::cout << "DAG before " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl; // std::cout << action_dag->dumpDAG() << std::endl; QueryTreeActionsVisitor visitor(action_dag, getContext()); - auto projection_action_dag_nodes = visitor.visit(query_tree->getProjectionNode()); + auto projection_action_dag_nodes = visitor.visit(query_tree_typed.getProjectionNode()); size_t projection_action_dag_nodes_size = projection_action_dag_nodes.size(); // std::cout << "Projection action dag nodes size " << projection_action_dag_nodes_size << std::endl; @@ -531,7 +550,7 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() // std::cout << "DAG after " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl; // std::cout << action_dag->dumpDAG() << std::endl; - auto & projection_nodes = query_tree->getProjection().getNodes(); + auto & projection_nodes = query_tree_typed.getProjection().getNodes(); size_t projection_nodes_size = projection_nodes.size(); if (projection_nodes_size != projection_action_dag_nodes_size) @@ -576,29 +595,42 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() size_t max_streams = current_context->getSettingsRef().max_threads; SelectQueryInfo query_info; - query_info.original_query = query_ptr; - query_info.query = query_ptr; + query_info.original_query = query; + query_info.query = query; - auto from_stage = table_node->getStorage()->getQueryProcessingStage( - current_context, select_query_options.to_stage, table_node->getStorageSnapshot(), query_info); - - Names column_names(source_columns_set.begin(), source_columns_set.end()); - - if (column_names.empty() && table_node->getStorage()->getName() == "SystemOne") - column_names.push_back("dummy"); - - if (!column_names.empty()) - table_node->getStorage()->read( - query_plan, column_names, table_node->getStorageSnapshot(), query_info, getContext(), from_stage, max_block_size, max_streams); - - /// Create step which reads from empty source if storage has no data. - if (!query_plan.isInitialized()) + if (auto * table_node = query_tree_typed.getFrom()->as()) { - auto source_header = table_node->getStorageSnapshot()->getSampleBlockForColumns(column_names); - Pipe pipe(std::make_shared(source_header)); - auto read_from_pipe = std::make_unique(std::move(pipe)); - read_from_pipe->setStepDescription("Read from NullSource"); - query_plan.addStep(std::move(read_from_pipe)); + auto from_stage = table_node->getStorage()->getQueryProcessingStage( + current_context, select_query_options.to_stage, table_node->getStorageSnapshot(), query_info); + + Names column_names(source_columns_set.begin(), source_columns_set.end()); + + if (column_names.empty() && table_node->getStorage()->getName() == "SystemOne") + column_names.push_back("dummy"); + + if (!column_names.empty()) + table_node->getStorage()->read( + query_plan, column_names, table_node->getStorageSnapshot(), query_info, getContext(), from_stage, max_block_size, max_streams); + + /// Create step which reads from empty source if storage has no data. + if (!query_plan.isInitialized()) + { + auto source_header = table_node->getStorageSnapshot()->getSampleBlockForColumns(column_names); + Pipe pipe(std::make_shared(source_header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource"); + query_plan.addStep(std::move(read_from_pipe)); + } + } + else if (auto * query_node = query_tree_typed.getFrom()->as()) + { + InterpreterSelectQueryAnalyzer interpeter(query_tree_typed.getFrom(), select_query_options, getContext()); + interpeter.initializeQueryPlanIfNeeded(); + query_plan = std::move(interpeter.query_plan); + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only single table or query in FROM section are supported"); } auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), action_dag); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 4f26cef801f..561d4054f4e 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -13,10 +13,17 @@ namespace DB class InterpreterSelectQueryAnalyzer : public IInterpreter, public WithContext { public: + /// Initialize interpreter with query AST InterpreterSelectQueryAnalyzer( - const ASTPtr & query_ptr_, + const ASTPtr & query_, const SelectQueryOptions & select_query_options_, - ContextPtr context); + ContextPtr context_); + + /// Initialize interpreter with query tree after query analysis and others phases + InterpreterSelectQueryAnalyzer( + const QueryTreeNodePtr & query_tree_, + const SelectQueryOptions & select_query_options_, + ContextPtr context_); Block getSampleBlock(); @@ -27,10 +34,10 @@ public: private: void initializeQueryPlanIfNeeded(); - ASTPtr query_ptr; + ASTPtr query; + QueryTreeNodePtr query_tree; QueryPlan query_plan; SelectQueryOptions select_query_options; - QueryTreePassManager query_tree_pass_manager; }; } diff --git a/tests/queries/0_stateless/02345_analyzer_subqueries.reference b/tests/queries/0_stateless/02345_analyzer_subqueries.reference new file mode 100644 index 00000000000..1e70be9ef52 --- /dev/null +++ b/tests/queries/0_stateless/02345_analyzer_subqueries.reference @@ -0,0 +1,27 @@ +Scalar subqueries +1 +1 +0 +Value +(0,'Value') +Subqueries FROM section +1 +1 +1 +1 +1 +1 +0 Value +0 Value +2 +Subqueries CTE +1 +1 +1 +1 +1 +1 +1 +1 +2 +2 diff --git a/tests/queries/0_stateless/02345_analyzer_subqueries.sql b/tests/queries/0_stateless/02345_analyzer_subqueries.sql new file mode 100644 index 00000000000..3694028e5d2 --- /dev/null +++ b/tests/queries/0_stateless/02345_analyzer_subqueries.sql @@ -0,0 +1,51 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +SELECT 'Scalar subqueries'; + +SELECT (SELECT 1); +WITH 1 AS a SELECT (SELECT a); + +SELECT (SELECT id FROM test_table); +SELECT (SELECT value FROM test_table); +SELECT (SELECT id, value FROM test_table); + +SELECT 'Subqueries FROM section'; + +SELECT a FROM (SELECT 1 AS a) AS b; +SELECT b.a FROM (SELECT 1 AS a) AS b; + +SELECT a FROM (SELECT 1 AS a) AS b; +SELECT b.a FROM (SELECT 1 AS a) AS b; + +WITH 1 AS global_a SELECT a FROM (SELECT global_a AS a) AS b; +WITH 1 AS global_a SELECT b.a FROM (SELECT global_a AS a) AS b; + +SELECT * FROM (SELECT * FROM (SELECT * FROM test_table)); +SELECT * FROM (SELECT id, value FROM (SELECT * FROM test_table)); + +WITH 1 AS a SELECT (SELECT * FROM (SELECT * FROM (SELECT a + 1))) SETTINGS use_analyzer=1; + +SELECT 'Subqueries CTE'; + +WITH subquery AS (SELECT 1 AS a) SELECT * FROM subquery; +WITH subquery AS (SELECT 1 AS a) SELECT a FROM subquery; +WITH subquery AS (SELECT 1 AS a) SELECT subquery.a FROM subquery; +WITH subquery AS (SELECT 1 AS a) SELECT subquery.* FROM subquery; +WITH subquery AS (SELECT 1 AS a) SELECT subquery.* APPLY toString FROM subquery; +WITH subquery AS (SELECT 1 AS a) SELECT subquery_alias.a FROM subquery AS subquery_alias; +WITH subquery AS (SELECT 1 AS a) SELECT subquery_alias.* FROM subquery AS subquery_alias; +WITH subquery AS (SELECT 1 AS a) SELECT subquery_alias.* APPLY toString FROM subquery AS subquery_alias; + +WITH subquery_1 AS (SELECT 1 AS a), subquery_2 AS (SELECT 1 + subquery_1.a FROM subquery_1) SELECT * FROM subquery_2; +WITH subquery_1 AS (SELECT 1 AS a), subquery_2 AS (SELECT (1 + subquery_1.a) AS a FROM subquery_1) SELECT subquery_2.a FROM subquery_2; + +DROP TABLE test_table; From 603c4febc3a075ec8edd86f098f4d69f352330af Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 19 Jul 2022 12:54:45 +0200 Subject: [PATCH 057/188] Update explain query tree representation --- src/Analyzer/ColumnNode.cpp | 15 ++++--- src/Analyzer/ColumnNode.h | 6 +-- src/Analyzer/ColumnTransformers.cpp | 46 ++++++++++---------- src/Analyzer/ColumnTransformers.h | 12 ++--- src/Analyzer/ConstantNode.cpp | 12 +++-- src/Analyzer/ConstantNode.h | 4 +- src/Analyzer/FunctionNode.cpp | 18 +++++--- src/Analyzer/FunctionNode.h | 6 +-- src/Analyzer/IQueryTreeNode.cpp | 14 +++++- src/Analyzer/IQueryTreeNode.h | 30 ++++++++----- src/Analyzer/IdentifierNode.cpp | 11 +++-- src/Analyzer/IdentifierNode.h | 6 +-- src/Analyzer/LambdaNode.cpp | 24 ++++++---- src/Analyzer/LambdaNode.h | 6 +-- src/Analyzer/ListNode.cpp | 10 ++--- src/Analyzer/ListNode.h | 6 +-- src/Analyzer/MatcherNode.cpp | 17 ++++---- src/Analyzer/MatcherNode.h | 6 +-- src/Analyzer/MultiIfToIfPass.h | 2 +- src/Analyzer/QueryNode.cpp | 25 ++++++----- src/Analyzer/QueryNode.h | 4 +- src/Analyzer/TableNode.cpp | 11 +++-- src/Analyzer/TableNode.h | 6 +-- src/Analyzer/tests/gtest_query_tree_node.cpp | 7 +-- src/Interpreters/InterpreterExplainQuery.cpp | 29 ++++++------ src/Parsers/ASTExplainQuery.h | 4 +- src/Parsers/ParserExplainQuery.cpp | 2 +- 27 files changed, 191 insertions(+), 148 deletions(-) diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index 86b9f940e44..15a59770423 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -28,13 +28,18 @@ QueryTreeNodePtr ColumnNode::getColumnSource() const return lock; } -void ColumnNode::dumpTree(WriteBuffer & buffer, size_t indent) const +void ColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t indent) const { - buffer << std::string(indent, ' ') << "COLUMN "; - writePointerHex(this, buffer); - buffer << ' ' << column.name << " : " << column.type->getName() << " source "; + buffer << std::string(indent, ' ') << "COLUMN id: " << state.getNodeId(this); + + if (hasAlias()) + buffer << ", alias: " << getAlias(); + + buffer << ", column_name: " << column.name << ", result_type: " << column.type->getName(); + auto column_source_ptr = column_source.lock(); - writePointerHex(column_source_ptr.get(), buffer); + if (column_source_ptr) + buffer << ", source_id: " << state.getNodeId(column_source_ptr.get()); } bool ColumnNode::isEqualImpl(const IQueryTreeNode & rhs) const diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index 8300b2485bb..2b182da2ae3 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -67,8 +67,6 @@ public: return QueryTreeNodeType::COLUMN; } - void dumpTree(WriteBuffer & buffer, size_t indent) const override; - String getName() const override { return column.name; @@ -79,11 +77,13 @@ public: return column.type; } -protected: + void dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t indent) const override; + bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & hash_state) const override; +protected: ASTPtr toASTImpl() const override; QueryTreeNodePtr cloneImpl() const override; diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp index c6a3945950b..9bcc1bb2d02 100644 --- a/src/Analyzer/ColumnTransformers.cpp +++ b/src/Analyzer/ColumnTransformers.cpp @@ -57,16 +57,15 @@ ApplyColumnTransformerNode::ApplyColumnTransformerNode(QueryTreeNodePtr expressi children[expression_child_index] = std::move(expression_node_); } -void ApplyColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) const +void ApplyColumnTransformerNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { - buffer << std::string(indent, ' ') << "APPLY COLUMN TRANSFORMER "; - writePointerHex(this, buffer); - buffer << ' ' << toString(apply_transformer_type) << '\n'; + buffer << std::string(indent, ' ') << "APPLY COLUMN TRANSFORMER id: " << format_state.getNodeId(this); + buffer << ", apply_transformer_type: " << toString(apply_transformer_type); - buffer << std::string(indent + 2, ' ') << "EXPRESSION" << '\n'; + buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION" << '\n'; const auto & expression_node = getExpressionNode(); - expression_node->dumpTree(buffer, indent + 4); + expression_node->dumpTreeImpl(buffer, format_state, indent + 4); } bool ApplyColumnTransformerNode::isEqualImpl(const IQueryTreeNode & rhs) const @@ -136,25 +135,28 @@ const char * toString(ExceptColumnTransformerType type) } } -void ExceptColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) const +void ExceptColumnTransformerNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { - buffer << std::string(indent, ' ') << "EXCEPT COLUMN TRANSFORMER "; - writePointerHex(this, buffer); - buffer << ' ' << toString(except_transformer_type) << ' '; + buffer << std::string(indent, ' ') << "EXCEPT COLUMN TRANSFORMER id: " << format_state.getNodeId(this); + buffer << ", except_transformer_type: " << toString(except_transformer_type); if (column_matcher) { - buffer << column_matcher->pattern(); + buffer << ", pattern: " << column_matcher->pattern(); return; } - - size_t except_column_names_size = except_column_names.size(); - for (size_t i = 0; i < except_column_names_size; ++i) + else { - buffer << except_column_names[i]; + buffer << ", identifiers: "; - if (i + 1 != except_column_names_size) - buffer << ", "; + size_t except_column_names_size = except_column_names.size(); + for (size_t i = 0; i < except_column_names_size; ++i) + { + buffer << except_column_names[i]; + + if (i + 1 != except_column_names_size) + buffer << ", "; + } } } @@ -259,15 +261,13 @@ QueryTreeNodePtr ReplaceColumnTransformerNode::findReplacementExpression(const s return replacement_expressions_nodes[replacement_index]; } -void ReplaceColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) const +void ReplaceColumnTransformerNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { - buffer << std::string(indent, ' ') << "REPLACE TRANSFORMER "; - writePointerHex(this, buffer); - buffer << '\n'; + buffer << std::string(indent, ' ') << "REPLACE COLUMN TRANSFORMER id: " << format_state.getNodeId(this); auto & replacements_nodes = getReplacements().getNodes(); size_t replacements_size = replacements_nodes.size(); - buffer << std::string(indent + 2, ' ') << "REPLACEMENTS " << replacements_size << '\n'; + buffer << '\n' << std::string(indent + 2, ' ') << "REPLACEMENTS " << replacements_size << '\n'; for (size_t i = 0; i < replacements_size; ++i) { @@ -275,7 +275,7 @@ void ReplaceColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) buffer << std::string(indent + 4, ' ') << "REPLACEMENT NAME " << replacement_name; buffer << " EXPRESSION" << '\n'; const auto & expression_node = replacements_nodes[i]; - expression_node->dumpTree(buffer, indent + 6); + expression_node->dumpTreeImpl(buffer, format_state, indent + 6); if (i + 1 != replacements_size) buffer << '\n'; diff --git a/src/Analyzer/ColumnTransformers.h b/src/Analyzer/ColumnTransformers.h index e5e82033ec8..5956824ebd7 100644 --- a/src/Analyzer/ColumnTransformers.h +++ b/src/Analyzer/ColumnTransformers.h @@ -129,13 +129,13 @@ public: return ColumnTransfomerType::APPLY; } - void dumpTree(WriteBuffer & buffer, size_t indent) const override; + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; -protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override; +protected: ASTPtr toASTImpl() const override; QueryTreeNodePtr cloneImpl() const override; @@ -218,13 +218,13 @@ public: return ColumnTransfomerType::EXCEPT; } - void dumpTree(WriteBuffer & buffer, size_t indent) const override; + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; -protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override; +protected: ASTPtr toASTImpl() const override; QueryTreeNodePtr cloneImpl() const override; @@ -294,13 +294,13 @@ public: */ QueryTreeNodePtr findReplacementExpression(const std::string & expression_name); - void dumpTree(WriteBuffer & buffer, size_t indent) const override; + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; -protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override; +protected: ASTPtr toASTImpl() const override; QueryTreeNodePtr cloneImpl() const override; diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index a6c6189d03a..92d9b56d5cf 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -26,11 +26,15 @@ ConstantNode::ConstantNode(Field value_) , type(applyVisitor(FieldToDataType(), value)) {} -void ConstantNode::dumpTree(WriteBuffer & buffer, size_t indent) const +void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { - buffer << std::string(indent, ' ') << "CONSTANT "; - writePointerHex(this, buffer); - buffer << ' ' << value.dump() << " : " << type->getName(); + buffer << std::string(indent, ' ') << "CONSTANT id: " << format_state.getNodeId(this); + + if (hasAlias()) + buffer << ", alias: " << getAlias(); + + buffer << ", value: " << value.dump(); + buffer << ", result_type: " << type->getName(); } bool ConstantNode::isEqualImpl(const IQueryTreeNode & rhs) const diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index ed605f03256..02fd6a2c65f 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -34,8 +34,6 @@ public: return QueryTreeNodeType::CONSTANT; } - void dumpTree(WriteBuffer & buffer, size_t indent) const override; - String getName() const override { return value_string_dump; @@ -47,6 +45,8 @@ public: } protected: + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & hash_state) const override; diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index 30e2a5a8dd0..195e9306cdb 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -30,24 +30,30 @@ void FunctionNode::resolveAsAggregateFunction(AggregateFunctionPtr aggregate_fun function_name = aggregate_function->getName(); } -void FunctionNode::dumpTree(WriteBuffer & buffer, size_t indent) const +void FunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { - buffer << std::string(indent, ' ') << "FUNCTION "; - writePointerHex(this, buffer); - buffer << ' ' << function_name << (result_type ? (" : " + result_type->getName()) : ""); + buffer << std::string(indent, ' ') << "FUNCTION id: " << format_state.getNodeId(this); + + if (hasAlias()) + buffer << ", alias: " << getAlias(); + + buffer << ", function_name: " << function_name; + + if (result_type) + buffer << ", result_type: " + result_type->getName(); const auto & parameters = getParameters(); if (!parameters.getNodes().empty()) { buffer << '\n' << std::string(indent + 2, ' ') << "PARAMETERS\n"; - parameters.dumpTree(buffer, indent + 4); + parameters.dumpTreeImpl(buffer, format_state, indent + 4); } const auto & arguments = getArguments(); if (!arguments.getNodes().empty()) { buffer << '\n' << std::string(indent + 2, ' ') << "ARGUMENTS\n"; - arguments.dumpTree(buffer, indent + 4); + arguments.dumpTreeImpl(buffer, format_state, indent + 4); } } diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index 657b81811df..2020afe89df 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -152,8 +152,6 @@ public: return QueryTreeNodeType::FUNCTION; } - void dumpTree(WriteBuffer & buffer, size_t indent) const override; - DataTypePtr getResultType() const override { return result_type; @@ -161,11 +159,13 @@ public: String getName() const override; -protected: + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & hash_state) const override; +protected: ASTPtr toASTImpl() const override; QueryTreeNodePtr cloneImpl() const override; diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 710ebb01561..01977651180 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -38,10 +38,22 @@ const char * toString(QueryTreeNodeType type) String IQueryTreeNode::dumpTree() const { WriteBufferFromOwnString buff; - dumpTree(buff, 0); + dumpTree(buff); return buff.str(); } +size_t IQueryTreeNode::FormatState::getNodeId(const IQueryTreeNode * node) +{ + auto [it, _] = node_to_id.emplace(node, node_to_id.size()); + return it->second; +} + +void IQueryTreeNode::dumpTree(WriteBuffer & buffer) const +{ + FormatState state; + dumpTreeImpl(buffer, state, 0); +} + bool IQueryTreeNode::isEqual(const IQueryTreeNode & rhs) const { if (getNodeType() != rhs.getNodeType()) diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 1df296ab3de..17a7a481789 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -84,13 +84,7 @@ public: String dumpTree() const; /// Dump query tree to buffer - void dumpTree(WriteBuffer & buffer) const - { - dumpTree(buffer, 0); - } - - /// Dump query tree to buffer starting with indent - virtual void dumpTree(WriteBuffer & buffer, size_t indent) const = 0; + void dumpTree(WriteBuffer & buffer) const; /** Is tree equal to other tree with node root. * @@ -179,7 +173,7 @@ public: if (original_ast) return formatOriginalASTForErrorMessage(); - return formatConvertedASTForErrorMessage(); + return formatConvertedASTForErrorMessage(); } /// Get query tree node children @@ -194,11 +188,26 @@ public: return children; } + class FormatState + { + public: + size_t getNodeId(const IQueryTreeNode * node); + + private: + std::unordered_map node_to_id; + }; + + /** Dump query tree to buffer starting with indent. + * + * Node must also dump its children. + */ + virtual void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const = 0; + /** Subclass must compare its internal state with rhs node and do not compare its children with rhs node children. * Caller must compare node and rhs node children. * * This method is not protected because if subclass node has weak pointers to other query tree nodes it must use it - * as part of its isEqualImpl method. In child classes this method should be protected. + * as part of its isEqualImpl method. */ virtual bool isEqualImpl(const IQueryTreeNode & rhs) const = 0; @@ -206,11 +215,12 @@ public: * Caller must update tree hash for node children. * * This method is not protected because if subclass node has weak pointers to other query tree nodes it must use it - * as part of its updateTreeHashImpl method. In child classes this method should be protected. + * as part of its updateTreeHashImpl method. */ virtual void updateTreeHashImpl(HashState & hash_state) const = 0; protected: + /** Subclass node must convert itself to AST. * Subclass must convert children to AST. */ diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp index 510a9cec93c..5275ec8c586 100644 --- a/src/Analyzer/IdentifierNode.cpp +++ b/src/Analyzer/IdentifierNode.cpp @@ -10,11 +10,14 @@ namespace DB { -void IdentifierNode::dumpTree(WriteBuffer & buffer, size_t indent) const +void IdentifierNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { - buffer << std::string(indent, ' ') << "IDENTIFIER "; - writePointerHex(this, buffer); - buffer << ' ' << identifier.getFullName(); + buffer << std::string(indent, ' ') << "IDENTIFIER id: " << format_state.getNodeId(this); + + if (hasAlias()) + buffer << ", alias: " << getAlias(); + + buffer << ", identifier: " << identifier.getFullName(); } bool IdentifierNode::isEqualImpl(const IQueryTreeNode & rhs) const diff --git a/src/Analyzer/IdentifierNode.h b/src/Analyzer/IdentifierNode.h index 38f909173f4..06bde3d4e0a 100644 --- a/src/Analyzer/IdentifierNode.h +++ b/src/Analyzer/IdentifierNode.h @@ -32,18 +32,18 @@ public: return QueryTreeNodeType::IDENTIFIER; } - void dumpTree(WriteBuffer & buffer, size_t indent) const override; - String getName() const override { return identifier.getFullName(); } -protected: + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & state) const override; +protected: ASTPtr toASTImpl() const override; QueryTreeNodePtr cloneImpl() const override; diff --git a/src/Analyzer/LambdaNode.cpp b/src/Analyzer/LambdaNode.cpp index bd0d9f117dc..52778b30c5d 100644 --- a/src/Analyzer/LambdaNode.cpp +++ b/src/Analyzer/LambdaNode.cpp @@ -27,21 +27,27 @@ LambdaNode::LambdaNode(Names argument_names_, QueryTreeNodePtr expression_) children[expression_child_index] = std::move(expression_); } -void LambdaNode::dumpTree(WriteBuffer & buffer, size_t indent) const +void LambdaNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { auto result_type = getExpression()->getResultType(); - buffer << std::string(indent, ' ') << "LAMBDA "; - writePointerHex(this, buffer); - buffer << (result_type ? (" : " + result_type->getName()) : "") << '\n'; + buffer << std::string(indent, ' ') << "LAMBDA id: " << format_state.getNodeId(this); - buffer << std::string(indent + 2, ' ') << "ARGUMENTS " << '\n'; - getArguments().dumpTree(buffer, indent + 4); + if (hasAlias()) + buffer << ", alias: " << getAlias(); - buffer << '\n'; + if (result_type) + buffer << ", result_type: " << result_type->getName(); - buffer << std::string(indent + 2, ' ') << "EXPRESSION " << '\n'; - getExpression()->dumpTree(buffer, indent + 4); + const auto & arguments = getArguments(); + if (!arguments.getNodes().empty()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "ARGUMENTS " << '\n'; + getArguments().dumpTreeImpl(buffer, format_state, indent + 4); + } + + buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION " << '\n'; + getExpression()->dumpTreeImpl(buffer, format_state, indent + 4); } String LambdaNode::getName() const diff --git a/src/Analyzer/LambdaNode.h b/src/Analyzer/LambdaNode.h index a49bf780e12..dc005f2b120 100644 --- a/src/Analyzer/LambdaNode.h +++ b/src/Analyzer/LambdaNode.h @@ -88,8 +88,6 @@ public: return QueryTreeNodeType::LAMBDA; } - void dumpTree(WriteBuffer & buffer, size_t indent) const override; - String getName() const override; DataTypePtr getResultType() const override @@ -97,11 +95,13 @@ public: return getExpression()->getResultType(); } -protected: + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & state) const override; +protected: ASTPtr toASTImpl() const override; QueryTreeNodePtr cloneImpl() const override; diff --git a/src/Analyzer/ListNode.cpp b/src/Analyzer/ListNode.cpp index e7f14c62ac2..ffd72a3990e 100644 --- a/src/Analyzer/ListNode.cpp +++ b/src/Analyzer/ListNode.cpp @@ -11,17 +11,17 @@ namespace DB { -void ListNode::dumpTree(WriteBuffer & buffer, size_t indent) const +void ListNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { + buffer << std::string(indent, ' ') << "LIST id: " << format_state.getNodeId(this); + size_t children_size = children.size(); - buffer << std::string(indent, ' ') << "LIST "; - writePointerHex(this, buffer); - buffer << ' ' << children_size << '\n'; + buffer << ", nodes: " << children_size << '\n'; for (size_t i = 0; i < children_size; ++i) { const auto & node = children[i]; - node->dumpTree(buffer, indent + 2); + node->dumpTreeImpl(buffer, format_state, indent + 2); if (i + 1 != children_size) buffer << '\n'; diff --git a/src/Analyzer/ListNode.h b/src/Analyzer/ListNode.h index 029023b9d8a..413dc964612 100644 --- a/src/Analyzer/ListNode.h +++ b/src/Analyzer/ListNode.h @@ -33,15 +33,15 @@ public: return QueryTreeNodeType::LIST; } - void dumpTree(WriteBuffer & buffer, size_t indent) const override; - String getName() const override; -protected: + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState &) const override; +protected: ASTPtr toASTImpl() const override; QueryTreeNodePtr cloneImpl() const override; diff --git a/src/Analyzer/MatcherNode.cpp b/src/Analyzer/MatcherNode.cpp index af5b0de4990..3aa70557987 100644 --- a/src/Analyzer/MatcherNode.cpp +++ b/src/Analyzer/MatcherNode.cpp @@ -119,23 +119,22 @@ bool MatcherNode::isMatchingColumn(const std::string & column_name) return columns_identifiers_set.find(column_name) != columns_identifiers_set.end(); } -void MatcherNode::dumpTree(WriteBuffer & buffer, size_t indent) const +void MatcherNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { - buffer << std::string(indent, ' ') << "MATCHER "; - writePointerHex(this, buffer); + buffer << std::string(indent, ' ') << "MATCHER id: " << format_state.getNodeId(this); + + buffer << ", matcher_type: " << toString(matcher_type); if (!qualified_identifier.empty()) - buffer << ' ' << qualified_identifier.getFullName(); - - buffer << ' ' << toString(matcher_type); + buffer << ", qualified_identifier: " << qualified_identifier.getFullName(); if (columns_matcher) { - buffer << ' ' << columns_matcher->pattern(); + buffer << ", columns_pattern: " << columns_matcher->pattern(); } else if (matcher_type == MatcherNodeType::COLUMNS_LIST) { - buffer << ' '; + buffer << ", columns_identifiers: "; size_t columns_identifiers_size = columns_identifiers.size(); for (size_t i = 0; i < columns_identifiers_size; ++i) { @@ -150,7 +149,7 @@ void MatcherNode::dumpTree(WriteBuffer & buffer, size_t indent) const if (!column_transformers_list.getNodes().empty()) { buffer << '\n'; - column_transformers_list.dumpTree(buffer, indent + 2); + column_transformers_list.dumpTreeImpl(buffer, format_state, indent + 2); } } diff --git a/src/Analyzer/MatcherNode.h b/src/Analyzer/MatcherNode.h index 1d9d77d9b36..00024801091 100644 --- a/src/Analyzer/MatcherNode.h +++ b/src/Analyzer/MatcherNode.h @@ -141,15 +141,15 @@ public: return QueryTreeNodeType::ASTERISK; } - void dumpTree(WriteBuffer & buffer, size_t indent) const override; - String getName() const override; -protected: + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & hash_state) const override; +protected: ASTPtr toASTImpl() const override; QueryTreeNodePtr cloneImpl() const override; diff --git a/src/Analyzer/MultiIfToIfPass.h b/src/Analyzer/MultiIfToIfPass.h index fc4859d1769..619a4b965c7 100644 --- a/src/Analyzer/MultiIfToIfPass.h +++ b/src/Analyzer/MultiIfToIfPass.h @@ -15,7 +15,7 @@ public: String getName() override { return "MultiIfToIf"; } - String getDescription() override { return "Optimize multiIf to if for single argument"; } + String getDescription() override { return "Optimize multiIf to if for single argument."; } void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 67a66644eb4..a9584752e6d 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -22,44 +22,45 @@ QueryNode::QueryNode() children[projection_child_index] = std::make_shared(); } -void QueryNode::dumpTree(WriteBuffer & buffer, size_t indent) const +void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { - buffer << std::string(indent, ' ') << "QUERY "; - writePointerHex(this, buffer); + buffer << std::string(indent, ' ') << "QUERY id: " << format_state.getNodeId(this); if (hasAlias()) - buffer << " alias : " << getAlias(); + buffer << ", alias: " << getAlias(); - buffer << " subquery : " << is_subquery; - buffer << " cte : " << is_cte; - buffer << " cte_name : " << cte_name; + buffer << ", is_subquery: " << is_subquery; + buffer << ", is_cte: " << is_cte; + + if (!cte_name.empty()) + buffer << ", cte_name: " << cte_name; if (!getWith().getNodes().empty()) { buffer << '\n' << std::string(indent + 2, ' ') << "WITH\n"; - getWith().dumpTree(buffer, indent + 4); + getWith().dumpTreeImpl(buffer, format_state, indent + 4); } buffer << '\n'; buffer << std::string(indent + 2, ' ') << "PROJECTION\n"; - getProjection().dumpTree(buffer, indent + 4); + getProjection().dumpTreeImpl(buffer, format_state, indent + 4); if (getFrom()) { buffer << '\n' << std::string(indent + 2, ' ') << "JOIN TREE\n"; - getFrom()->dumpTree(buffer, indent + 4); + getFrom()->dumpTreeImpl(buffer, format_state, indent + 4); } if (getPrewhere()) { buffer << '\n' << std::string(indent + 2, ' ') << "PREWHERE\n"; - getPrewhere()->dumpTree(buffer, indent + 4); + getPrewhere()->dumpTreeImpl(buffer, format_state, indent + 4); } if (getWhere()) { buffer << '\n' << std::string(indent + 2, ' ') << "WHERE\n"; - getWhere()->dumpTree(buffer, indent + 4); + getWhere()->dumpTreeImpl(buffer, format_state, indent + 4); } } diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index b8d0d6a0d88..6d08bf0642f 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -122,13 +122,13 @@ public: return QueryTreeNodeType::QUERY; } - void dumpTree(WriteBuffer & buffer, size_t indent) const override; + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; -protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState &) const override; +protected: ASTPtr toASTImpl() const override; QueryTreeNodePtr cloneImpl() const override; diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index 647ef9eaf2e..65dbccfd27a 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -21,11 +21,14 @@ TableNode::TableNode(StoragePtr storage_, ContextPtr context) { } -void TableNode::dumpTree(WriteBuffer & buffer, size_t indent) const +void TableNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { - buffer << std::string(indent, ' ') << "TABLE "; - writePointerHex(this, buffer); - buffer << ' ' << storage_id.getFullNameNotQuoted(); + buffer << std::string(indent, ' ') << "TABLE id: " << format_state.getNodeId(this); + + if (hasAlias()) + buffer << ", alias: " << getAlias(); + + buffer << ", table_name: " << storage_id.getFullNameNotQuoted(); } bool TableNode::isEqualImpl(const IQueryTreeNode & rhs) const diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index a195d5c2df4..25f6529d489 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -66,15 +66,15 @@ public: return QueryTreeNodeType::TABLE; } - void dumpTree(WriteBuffer & buffer, size_t indent) const override; - String getName() const override; -protected: + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & state) const override; +protected: ASTPtr toASTImpl() const override; QueryTreeNodePtr cloneImpl() const override; diff --git a/src/Analyzer/tests/gtest_query_tree_node.cpp b/src/Analyzer/tests/gtest_query_tree_node.cpp index 56483614bcd..0940aa5861d 100644 --- a/src/Analyzer/tests/gtest_query_tree_node.cpp +++ b/src/Analyzer/tests/gtest_query_tree_node.cpp @@ -16,10 +16,8 @@ public: return QueryTreeNodeType::TABLE; } - void dumpTree(WriteBuffer & buffer, size_t indent) const override + void dumpTreeImpl(WriteBuffer &, FormatState &, size_t) const override { - (void)(buffer); - (void)(indent); } bool isEqualImpl(const IQueryTreeNode &) const override @@ -27,9 +25,8 @@ public: return true; } - void updateTreeHashImpl(HashState & hash_state) const override + void updateTreeHashImpl(HashState &) const override { - (void)(hash_state); } ASTPtr toASTImpl() const override diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index dbc742c9cb3..baba2214333 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -165,15 +165,17 @@ struct QueryASTSettings struct QueryTreeSettings { bool run_passes = false; + bool dump_passes = false; + bool dump_ast = false; Int64 passes = -1; - bool ast = false; - constexpr static char name[] = "QUERYTREE"; + constexpr static char name[] = "QUERY TREE"; std::unordered_map> boolean_settings = { {"run_passes", run_passes}, - {"ast", ast} + {"dump_passes", dump_passes}, + {"dump_ast", dump_ast} }; std::unordered_map> integer_settings = @@ -381,30 +383,25 @@ QueryPipeline InterpreterExplainQuery::executeImpl() auto query_tree_pass_manager = QueryTreePassManager(getContext()); addQueryTreePasses(query_tree_pass_manager); - if (settings.passes < 0) + size_t pass_index = settings.passes < 0 ? query_tree_pass_manager.getPasses().size() : static_cast(settings.passes); + + if (settings.dump_passes) { - query_tree_pass_manager.dump(buf); - buf << '\n'; - query_tree_pass_manager.run(query_tree); - } - else - { - size_t pass_index = static_cast(settings.passes); query_tree_pass_manager.dump(buf, pass_index); if (pass_index > 0) buf << '\n'; - - query_tree_pass_manager.run(query_tree, pass_index); } - query_tree->dumpTree(buf, 0); + query_tree_pass_manager.run(query_tree, pass_index); + + query_tree->dumpTree(buf); } else { - query_tree->dumpTree(buf, 0); + query_tree->dumpTree(buf); } - if (settings.ast) + if (settings.dump_ast) { buf << '\n'; query_tree->toAST()->format(IAST::FormatSettings(buf, false)); diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index f802751b0ec..156ffdeacb9 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -15,7 +15,7 @@ public: { ParsedAST, /// 'EXPLAIN AST SELECT ...' AnalyzedSyntax, /// 'EXPLAIN SYNTAX SELECT ...' - QueryTree, /// 'EXPLAIN QUERYTREE SELECT ...' + QueryTree, /// 'EXPLAIN QUERY TREE SELECT ...' QueryPlan, /// 'EXPLAIN SELECT ...' QueryPipeline, /// 'EXPLAIN PIPELINE ...' QueryEstimates, /// 'EXPLAIN ESTIMATE ...' @@ -110,7 +110,7 @@ private: { case ParsedAST: return "EXPLAIN AST"; case AnalyzedSyntax: return "EXPLAIN SYNTAX"; - case QueryTree: return "EXPLAIN QUERYTREE"; + case QueryTree: return "EXPLAIN QUERY TREE"; case QueryPlan: return "EXPLAIN"; case QueryPipeline: return "EXPLAIN PIPELINE"; case QueryEstimates: return "EXPLAIN ESTIMATE"; diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index 52d82a29118..62e23202031 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -19,7 +19,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_ast("AST"); ParserKeyword s_explain("EXPLAIN"); ParserKeyword s_syntax("SYNTAX"); - ParserKeyword s_query_tree("QUERYTREE"); + ParserKeyword s_query_tree("QUERY TREE"); ParserKeyword s_pipeline("PIPELINE"); ParserKeyword s_plan("PLAN"); ParserKeyword s_estimates("ESTIMATE"); From 425eb16514418035412f23a63d0986a173e4cb95 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 19 Jul 2022 15:25:15 +0200 Subject: [PATCH 058/188] Added EXPLAIN QUERY TREE tests --- .../02366_explain_query_tree.reference | 95 +++++++++++++++++++ .../0_stateless/02366_explain_query_tree.sql | 38 ++++++++ 2 files changed, 133 insertions(+) create mode 100644 tests/queries/0_stateless/02366_explain_query_tree.reference create mode 100644 tests/queries/0_stateless/02366_explain_query_tree.sql diff --git a/tests/queries/0_stateless/02366_explain_query_tree.reference b/tests/queries/0_stateless/02366_explain_query_tree.reference new file mode 100644 index 00000000000..42cc6777e42 --- /dev/null +++ b/tests/queries/0_stateless/02366_explain_query_tree.reference @@ -0,0 +1,95 @@ +QUERY id: 0, is_subquery: 0, is_cte: 0 + PROJECTION + LIST id: 1, nodes: 1 + CONSTANT id: 2, value: UInt64_1, result_type: UInt8 + JOIN TREE + IDENTIFIER id: 3, identifier: system.one +-- +QUERY id: 0, is_subquery: 0, is_cte: 0 + PROJECTION + LIST id: 1, nodes: 2 + IDENTIFIER id: 2, identifier: id + IDENTIFIER id: 3, identifier: value + JOIN TREE + IDENTIFIER id: 4, identifier: test_table +-- +QUERY id: 0, is_subquery: 0, is_cte: 0 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 + COLUMN id: 4, column_name: value, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.test_table +-- +QUERY id: 0, is_subquery: 0, is_cte: 0 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: arrayMap + ARGUMENTS + LIST id: 3, nodes: 2 + LAMBDA id: 4 + ARGUMENTS + LIST id: 5, nodes: 1 + IDENTIFIER id: 6, identifier: x + EXPRESSION + FUNCTION id: 7, function_name: plus + ARGUMENTS + LIST id: 8, nodes: 2 + IDENTIFIER id: 9, identifier: x + IDENTIFIER id: 10, identifier: id + CONSTANT id: 11, value: Array_[UInt64_1, UInt64_2, UInt64_3], result_type: Array(UInt8) + JOIN TREE + IDENTIFIER id: 12, identifier: test_table +-- +QUERY id: 0, is_subquery: 0, is_cte: 0 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: arrayMap, result_type: Array(UInt16) + ARGUMENTS + LIST id: 3, nodes: 2 + LAMBDA id: 4, result_type: UInt16 + ARGUMENTS + LIST id: 5, nodes: 1 + COLUMN id: 6, column_name: x, result_type: UInt8, source_id: 4 + EXPRESSION + FUNCTION id: 7, function_name: plus, result_type: UInt16 + ARGUMENTS + LIST id: 8, nodes: 2 + COLUMN id: 6, column_name: x, result_type: UInt8, source_id: 4 + CONSTANT id: 9, value: UInt64_1, result_type: UInt8 + CONSTANT id: 10, value: Array_[UInt64_1, UInt64_2, UInt64_3], result_type: Array(UInt8) + JOIN TREE + TABLE id: 11, table_name: default.test_table +-- +QUERY id: 0, is_subquery: 0, is_cte: 0 + WITH + LIST id: 1, nodes: 1 + LAMBDA id: 2, alias: lambda + ARGUMENTS + LIST id: 3, nodes: 1 + IDENTIFIER id: 4, identifier: x + EXPRESSION + FUNCTION id: 5, function_name: plus + ARGUMENTS + LIST id: 6, nodes: 2 + IDENTIFIER id: 7, identifier: x + CONSTANT id: 8, value: UInt64_1, result_type: UInt8 + PROJECTION + LIST id: 9, nodes: 1 + FUNCTION id: 10, function_name: lambda + ARGUMENTS + LIST id: 11, nodes: 1 + IDENTIFIER id: 12, identifier: id + JOIN TREE + IDENTIFIER id: 13, identifier: test_table +-- +QUERY id: 0, is_subquery: 0, is_cte: 0 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: plus, result_type: UInt64 + ARGUMENTS + LIST id: 3, nodes: 2 + COLUMN id: 4, column_name: id, result_type: UInt64, source_id: 5 + CONSTANT id: 6, value: UInt64_1, result_type: UInt8 + JOIN TREE + TABLE id: 5, table_name: default.test_table diff --git a/tests/queries/0_stateless/02366_explain_query_tree.sql b/tests/queries/0_stateless/02366_explain_query_tree.sql new file mode 100644 index 00000000000..23df6d12121 --- /dev/null +++ b/tests/queries/0_stateless/02366_explain_query_tree.sql @@ -0,0 +1,38 @@ +SET use_analyzer = 1; + +EXPLAIN QUERY TREE SELECT 1; + +SELECT '--'; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +EXPLAIN QUERY TREE SELECT id, value FROM test_table; + +SELECT '--'; + +EXPLAIN QUERY TREE run_passes = 1 SELECT id, value FROM test_table; + +SELECT '--'; + +EXPLAIN QUERY TREE SELECT arrayMap(x -> x + id, [1, 2, 3]) FROM test_table; + +SELECT '--'; + +EXPLAIN QUERY TREE run_passes = 1 SELECT arrayMap(x -> x + 1, [1, 2, 3]) FROM test_table; + +SELECT '--'; + +EXPLAIN QUERY TREE WITH x -> x + 1 AS lambda SELECT lambda(id) FROM test_table; + +SELECT '--'; + +EXPLAIN QUERY TREE run_passes = 1 WITH x -> x + 1 AS lambda SELECT lambda(id) FROM test_table; + +DROP TABLE test_table; From 569fb36097536aad4549b0b8b119caad5548cf32 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 20 Jul 2022 12:34:29 +0200 Subject: [PATCH 059/188] Added support for table ALIAS columns --- src/Analyzer/ColumnNode.cpp | 23 ++ src/Analyzer/ColumnNode.h | 29 ++- src/Analyzer/QueryAnalysisPass.cpp | 214 +++++++++++++++--- .../InterpreterSelectQueryAnalyzer.cpp | 10 +- ...367_analyzer_table_alias_columns.reference | 3 + .../02367_analyzer_table_alias_columns.sql | 41 ++++ 6 files changed, 285 insertions(+), 35 deletions(-) create mode 100644 tests/queries/0_stateless/02367_analyzer_table_alias_columns.reference create mode 100644 tests/queries/0_stateless/02367_analyzer_table_alias_columns.sql diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index 15a59770423..28023f7328a 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -16,6 +16,21 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_) + : column(std::move(column_)) + , column_source(std::move(column_source_)) +{ + children.resize(1); +} + +ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodePtr alias_expression_node_, QueryTreeNodeWeakPtr column_source_) + : column(std::move(column_)) + , column_source(std::move(column_source_)) +{ + children.resize(1); + children[0] = std::move(alias_expression_node_); +} + QueryTreeNodePtr ColumnNode::getColumnSource() const { auto lock = column_source.lock(); @@ -40,6 +55,14 @@ void ColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t auto column_source_ptr = column_source.lock(); if (column_source_ptr) buffer << ", source_id: " << state.getNodeId(column_source_ptr.get()); + + const auto & alias_expression = getAliasExpression(); + + if (alias_expression) + { + buffer << '\n' << std::string(indent + 2, ' ') << "ALIAS EXPRESSION\n"; + alias_expression->dumpTreeImpl(buffer, state, indent + 4); + } } bool ColumnNode::isEqualImpl(const IQueryTreeNode & rhs) const diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index 2b182da2ae3..5a27e23c5cb 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -12,6 +12,8 @@ namespace DB * Column can be table expression, lambda, subquery. * Column source must be valid during column node lifetime. * + * For table ALIAS columns. Column node must contain ALIAS expression. + * * During query analysis pass identifier node is resolved into column. See IdentifierNode.h. * * Examples: @@ -19,6 +21,8 @@ namespace DB * SELECT lambda(x -> x + 1, [1,2,3]). x is identifier inside lambda that must be resolved to column node during query analysis pass. * * Column node is initialized with column name, type and column source weak pointer. + * In case of ALIAS column node is initialized with column name, type, alias expression and column source weak pointer. + * * Additional care must be taken during clone to repoint column source to another node if its necessary see IQueryTreeNode.h `clone` method. */ class ColumnNode; @@ -28,10 +32,10 @@ class ColumnNode final : public IQueryTreeNode { public: /// Construct column node with column name, type and column source weak pointer. - explicit ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_) - : column(std::move(column_)) - , column_source(std::move(column_source_)) - {} + ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_); + + /// Construct ALIAS column node with column name, type, column expression and column source weak pointer. + ColumnNode(NameAndTypePair column_, QueryTreeNodePtr alias_expression_node_, QueryTreeNodeWeakPtr column_source_); /// Get column const NameAndTypePair & getColumn() const @@ -51,6 +55,21 @@ public: return column.type; } + bool hasAliasExpression() const + { + return children[alias_expression_child_index] != nullptr; + } + + const QueryTreeNodePtr & getAliasExpression() const + { + return children[alias_expression_child_index]; + } + + QueryTreeNodePtr & getAliasExpression() + { + return children[alias_expression_child_index]; + } + /** Get column source. * If column source is not valid logical exception is thrown. */ @@ -93,6 +112,8 @@ protected: private: NameAndTypePair column; QueryTreeNodeWeakPtr column_source; + + static constexpr size_t alias_expression_child_index = 0; }; } diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 3b4986b2ff8..a34476fff5f 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -117,7 +117,7 @@ namespace ErrorCodes * Resolution starts with current scope. * 1. Try to resolve identifier from expression scope arguments. Lambda expression arguments are greatest priority. * 2. Try to resolve identifier from aliases. - * 3. Try to resolve identifier from tables if scope is query. + * 3. Try to resolve identifier from tables if scope is query, or if there are registered table columns in scope. * Steps 2 and 3 can be changed using prefer_column_name_to_alias setting. * If identifier could not be resolved in current scope, resolution must be continued in parent scopes. * 4. If it is table lookup, try to resolve identifier from CTE. @@ -151,7 +151,6 @@ namespace ErrorCodes * * TODO: Add expression name into query tree node. Example: SELECT plus(1, 1). Result: SELECT 2. Expression name of constant node should be 2. * TODO: Table identifiers with optional UUID. - * TODO: Table ALIAS columns * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. @@ -237,13 +236,24 @@ struct IdentifierLookupHash struct TableExpressionColumns { - NamesAndTypesList column_names_and_types; + std::unordered_map column_name_to_column_node; std::unordered_set column_identifier_first_parts; bool canBindIdentifier(IdentifierView identifier) { return column_identifier_first_parts.find(std::string(identifier.at(0))) != column_identifier_first_parts.end(); } + + void dump(WriteBuffer & buffer) + { + buffer << "Columns size " << column_name_to_column_node.size() << '\n'; + + for (auto & [column_name, column_node] : column_name_to_column_node) + { + buffer << "Column name " << column_name << " column node " << column_node->formatASTForErrorMessage(); + buffer << " is alias " << column_node->hasAliasExpression() << '\n'; + } + } }; class ExpressionsStack @@ -353,6 +363,9 @@ struct IdentifierResolveScope /// Alias name to table expression node std::unordered_map alias_name_to_table_expression_node; + /// Table column name to column node. Valid only during table ALIAS columns resolve. + std::unordered_map column_name_to_column_node; + /// CTE name to query node std::unordered_map cte_name_to_query_node; @@ -374,7 +387,12 @@ struct IdentifierResolveScope buffer << "Scope node " << scope_node->formatASTForErrorMessage() << '\n'; buffer << "Identifier lookup to node " << identifier_lookup_to_node.size() << '\n'; for (const auto & [identifier, node] : identifier_lookup_to_node) - buffer << "Identifier " << identifier.dump() << " node type " << node->getNodeTypeName() << " format " << node->formatASTForErrorMessage() << '\n'; + { + if (!node) + buffer << "Identifier " << identifier.dump() << " in resolve process\n"; + else + buffer << "Identifier " << identifier.dump() << " node type " << node->getNodeTypeName() << " format " << node->formatASTForErrorMessage() << '\n'; + } buffer << "Expression argument name to node " << expression_argument_name_to_node.size() << '\n'; for (const auto & [alias_name, node] : expression_argument_name_to_node) @@ -463,6 +481,8 @@ private: QueryTreeNodePtr tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, bool allow_to_check_parent_scopes = true); + QueryTreeNodePtr tryResolveIdentifierFromTableColumns(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); + QueryTreeNodePtr tryResolveIdentifierFromTables(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); // QueryTreeNodePtr tryResolveIdentifierInCurrentScope(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & current_scope); @@ -710,18 +730,25 @@ void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node) */ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) { - const auto & identifier_bind_part = identifier_lookup.identifier.front(); + auto it = scope.expression_argument_name_to_node.find(identifier_lookup.identifier.getFullName()); + bool resolve_full_identifier = it != scope.expression_argument_name_to_node.end(); - auto it = scope.expression_argument_name_to_node.find(identifier_bind_part); - if (it == scope.expression_argument_name_to_node.end()) + if (!resolve_full_identifier) + { + const auto & identifier_bind_part = identifier_lookup.identifier.front(); + + it = scope.expression_argument_name_to_node.find(identifier_bind_part); + if (it == scope.expression_argument_name_to_node.end()) + return {}; + } + + if (identifier_lookup.isExpressionLookup() && it->second->getNodeType() != QueryTreeNodeType::COLUMN && it->second->getNodeType() != QueryTreeNodeType::CONSTANT + && it->second->getNodeType() != QueryTreeNodeType::FUNCTION && it->second->getNodeType() != QueryTreeNodeType::QUERY) + return {}; + else if (identifier_lookup.isTableLookup() && it->second->getNodeType() != QueryTreeNodeType::TABLE && it->second->getNodeType() != QueryTreeNodeType::QUERY) return {}; - if (identifier_lookup.isExpressionLookup() && it->second->getNodeType() == QueryTreeNodeType::LAMBDA) - return {}; - else if (identifier_lookup.isTableLookup() && it->second->getNodeType() != QueryTreeNodeType::TABLE) - return {}; - - if (identifier_lookup.identifier.isCompound() && identifier_lookup.isExpressionLookup()) + if (!resolve_full_identifier && identifier_lookup.identifier.isCompound() && identifier_lookup.isExpressionLookup()) { auto nested_path = IdentifierView(identifier_lookup.identifier); nested_path.popFirst(); @@ -980,13 +1007,62 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier return result; } +/** Resolve identifier from table columns. + * + * 1. If table column nodes are empty or identifier is not expression lookup return nullptr. + * 2. If identifier full name match table column use column. Save information that we resolve identifier using full name. + * 3. Else if identifier binds to table column, use column. + * 4. Try to resolve column ALIAS expression if it exists. + * 5. If identifier was compound and was not resolved using full name during step 1 pop first part from identifier lookup and wrap column node + * using nested parts of identifier using `wrapExpressionNodeInTupleElement` function. + * This can be the case with compound ALIAS columns. + * Example: + * CREATE TABLE test_table (id UInt64, value Tuple(id UInt64, value String), alias_value ALIAS value.id) ENGINE=TinyLog; + */ +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableColumns(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +{ + if (scope.column_name_to_column_node.empty() || !identifier_lookup.isExpressionLookup()) + return {}; + + const auto & identifier = identifier_lookup.identifier; + auto it = scope.column_name_to_column_node.find(identifier.getFullName()); + bool full_column_name_match = it != scope.column_name_to_column_node.end(); + + if (!full_column_name_match) + { + it = scope.column_name_to_column_node.find(identifier_lookup.identifier[0]); + if (it == scope.column_name_to_column_node.end()) + return {}; + } + + if (it->second->hasAliasExpression()) + resolveExpressionNode(it->second->getAliasExpression(), scope, false /*allow_lambda_node*/); + + QueryTreeNodePtr result = it->second; + + if (!full_column_name_match && identifier.isCompound()) + { + auto nested_path = IdentifierView(identifier_lookup.identifier); + nested_path.popFirst(); + + auto tuple_element_result = wrapExpressionNodeInTupleElement(it->second, nested_path); + resolveFunction(tuple_element_result, scope); + + result = tuple_element_result; + } + + return result; +} + /** Resolve identifier from scope tables. * - * 1. If there is no table node in scope, or identifier is in function lookup context return nullptr. - * 2. If identifier is in table lookup context, check if it has 1 or 2 parts, otherwise throw exception. + * 1. If identifier is in function lookup context return nullptr. + * 2. Try to resolve identifier from table columns. + * 3. If there is no FROM section return nullptr. + * 4. If identifier is in table lookup context, check if it has 1 or 2 parts, otherwise throw exception. * If identifer has 2 parts try to match it with database_name and table_name. * If identifier has 1 part try to match it with table_name, then try to match it with table alias. - * 3. If identifier is in expression lookup context, we first need to bind identifier to some table column using identifier first part. + * 5. If identifier is in expression lookup context, we first need to bind identifier to some table column using identifier first part. * Start with identifier first part, if it match some column name in table try to get column with full identifier name. * TODO: Need to check if it is okay to throw exception if compound identifier first part bind to column but column is not valid. */ @@ -995,6 +1071,11 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL if (identifier_lookup.isFunctionLookup()) return {}; + /// Try to resolve identifier from table columns + + if (auto resolved_identifier = tryResolveIdentifierFromTableColumns(identifier_lookup, scope)) + return resolved_identifier; + auto * query_scope_node = scope.scope_node->as(); if (!query_scope_node || !query_scope_node->getFrom()) return {}; @@ -1019,7 +1100,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL std::string table_expression_name; - /// Valid only for table node std::string table_name; std::string database_name; @@ -1061,13 +1141,76 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL TableExpressionColumns storage_columns; if (table_node) - storage_columns.column_names_and_types = table_node->getStorageSnapshot()->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); - else if (query_node) - storage_columns.column_names_and_types = getQueryNodeColumns(from_node); - - for (auto & name_and_type_pair : storage_columns.column_names_and_types) { - Identifier column_name_identifier(name_and_type_pair.name); + const auto & storage_snapshot = table_node->getStorageSnapshot(); + + auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); + const auto & columns_description = storage_snapshot->metadata->getColumns(); + + std::vector> alias_columns_to_resolve; + std::unordered_map column_name_to_column_node; + column_name_to_column_node.reserve(column_names_and_types.size()); + + for (const auto & column_name_and_type : column_names_and_types) + { + const auto & column_default = columns_description.getDefault(column_name_and_type.name); + + if (column_default && column_default->kind == ColumnDefaultKind::Alias) + { + auto argument_node = std::make_shared(column_name_and_type, buildQueryTree(column_default->expression, context), from_node); + column_name_to_column_node.emplace(column_name_and_type.name, argument_node); + alias_columns_to_resolve.emplace_back(column_name_and_type.name, argument_node); + } + else + { + auto argument_node = std::make_shared(column_name_and_type, from_node); + column_name_to_column_node.emplace(column_name_and_type.name, argument_node); + } + } + + for (auto & [alias_column_to_resolve_name, alias_column_to_resolve] : alias_columns_to_resolve) + { + /** Alias column could be potentially resolved during resolve of other ALIAS column. + * Example: CREATE TABLE test_table (id UInt64, alias_value_1 ALIAS id + alias_value_2, alias_value_2 ALIAS id + 5) ENGINE=TinyLog; + * + * During resolve of alias_value_1, alias_value_2 column will be resolved. + */ + alias_column_to_resolve = column_name_to_column_node[alias_column_to_resolve_name]; + + IdentifierResolveScope alias_column_resolve_scope(alias_column_to_resolve, nullptr /*parent_scope*/); + alias_column_resolve_scope.column_name_to_column_node = std::move(column_name_to_column_node); + + /// Initialize aliases in alias column scope + ScopeAliasVisitorMatcher::Data data{alias_column_resolve_scope}; + ScopeAliasVisitorMatcher::Visitor visitor(data); + + visitor.visit(alias_column_to_resolve->getAliasExpression()); + + resolveExpressionNode(alias_column_resolve_scope.scope_node, alias_column_resolve_scope, false /*allow_lambda_expression*/); + + column_name_to_column_node = std::move(alias_column_resolve_scope.column_name_to_column_node); + column_name_to_column_node[alias_column_to_resolve_name] = alias_column_to_resolve; + } + + storage_columns.column_name_to_column_node = std::move(column_name_to_column_node); + } + else if (query_node) + { + auto column_names_and_types = getQueryNodeColumns(from_node); + storage_columns.column_name_to_column_node.reserve(column_names_and_types.size()); + + for (const auto & column_name_and_type : column_names_and_types) + { + auto column_node = std::make_shared(column_name_and_type, from_node); + storage_columns.column_name_to_column_node.emplace(column_name_and_type.name, column_node); + } + } + + storage_columns.column_identifier_first_parts.reserve(storage_columns.column_name_to_column_node.size()); + + for (auto & [column_name, _] : storage_columns.column_name_to_column_node) + { + Identifier column_name_identifier(column_name); storage_columns.column_identifier_first_parts.insert(column_name_identifier.at(0)); } @@ -1091,13 +1234,23 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL * Here there is no column with name test_subquery.compound_expression.value, and additional wrap in tuple element is required. */ - std::optional result_column; + QueryTreeNodePtr result_column; bool compound_identifier = identifier_view.getPartsSize() > 1; + bool match_full_identifier = false; - if (!table_node && compound_identifier) - result_column = storage_columns.column_names_and_types.tryGetByName(std::string(identifier_view.at(0))); + auto it = storage_columns.column_name_to_column_node.find(std::string(identifier_view.getFullName())); + if (it != storage_columns.column_name_to_column_node.end()) + { + match_full_identifier = true; + result_column = it->second; + } else - result_column = storage_columns.column_names_and_types.tryGetByName(std::string(identifier_view.getFullName())); + { + it = storage_columns.column_name_to_column_node.find(std::string(identifier_view.at(0))); + + if (it != storage_columns.column_name_to_column_node.end()) + result_column = it->second; + } if (!result_column) throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -1107,9 +1260,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL table_expression_name.empty() ? "" : " with name " + table_expression_name, scope.scope_node->formatASTForErrorMessage()); - QueryTreeNodePtr result_expression = std::make_shared(*result_column, from_node); + QueryTreeNodePtr result_expression = result_column; - if (!table_node && compound_identifier) + if (!match_full_identifier && compound_identifier) { IdentifierView nested_path(identifier_view); nested_path.popFirst(); @@ -2121,7 +2274,8 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes } else if (auto * column_function_node = node->as()) { - /// Already resolved + if (column_function_node->hasAliasExpression()) + resolveExpressionNode(column_function_node->getAliasExpression(), scope, false /*allow_lambda_expression*/); } else if (auto * lambda_node = node->as()) { diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 11456ead44d..63768d4f16e 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -428,6 +428,15 @@ public: if (!table_node) return; + /// Replace ALIAS column with expression + + if (column_node->hasAliasExpression()) + { + node = column_node->getAliasExpression(); + visit(node, data); + return; + } + data.source_columns_set.insert(column_node->getColumnName()); } @@ -437,7 +446,6 @@ public: } }; - using CollectSourceColumnsVisitor = CollectSourceColumnsMatcher::Visitor; InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( diff --git a/tests/queries/0_stateless/02367_analyzer_table_alias_columns.reference b/tests/queries/0_stateless/02367_analyzer_table_alias_columns.reference new file mode 100644 index 00000000000..1f4875e38c2 --- /dev/null +++ b/tests/queries/0_stateless/02367_analyzer_table_alias_columns.reference @@ -0,0 +1,3 @@ +0 6 5 +0 Value 2 +0 Value 2 diff --git a/tests/queries/0_stateless/02367_analyzer_table_alias_columns.sql b/tests/queries/0_stateless/02367_analyzer_table_alias_columns.sql new file mode 100644 index 00000000000..769ba769f4d --- /dev/null +++ b/tests/queries/0_stateless/02367_analyzer_table_alias_columns.sql @@ -0,0 +1,41 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + alias_value_1 ALIAS id + alias_value_2 + 1, + alias_value_2 ALIAS id + 5 +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0); + +SELECT id, alias_value_1, alias_value_2 FROM test_table; + +DROP TABLE test_table; + +CREATE TABLE test_table +( + id UInt64, + value String, + alias_value ALIAS ((id + 1) AS inside_value) + inside_value +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +SELECT id, value, alias_value FROM test_table; + +DROP TABLE test_table; + +CREATE TABLE test_table +( + id UInt64, + value String, + alias_value ALIAS ((id + 1) AS value) + value +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +SELECT id, value, alias_value FROM test_table; + +DROP TABLE test_table; From 25888cbac1d2458480dd5f6f8f7b651f16d03a04 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 20 Jul 2022 12:44:36 +0200 Subject: [PATCH 060/188] Fixed tests --- .../02341_analyzer_aliases_basics.reference | 4 ++-- .../02341_analyzer_aliases_basics.sql | 4 ++-- .../02342_analyzer_compound_types.reference | 24 +++++++++---------- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02341_analyzer_aliases_basics.reference b/tests/queries/0_stateless/02341_analyzer_aliases_basics.reference index ea60915f479..3733d6b6084 100644 --- a/tests/queries/0_stateless/02341_analyzer_aliases_basics.reference +++ b/tests/queries/0_stateless/02341_analyzer_aliases_basics.reference @@ -15,5 +15,5 @@ Alias conflict with identifier inside expression 1 3 Alias setting prefer_column_name_to_alias -0 0 -0 Value +0 +Value diff --git a/tests/queries/0_stateless/02341_analyzer_aliases_basics.sql b/tests/queries/0_stateless/02341_analyzer_aliases_basics.sql index 4abd8bc55f7..636936d875a 100644 --- a/tests/queries/0_stateless/02341_analyzer_aliases_basics.sql +++ b/tests/queries/0_stateless/02341_analyzer_aliases_basics.sql @@ -41,10 +41,10 @@ SELECT (id + 1 + 1 + 1 + id) AS id FROM test_table; SELECT 'Alias setting prefer_column_name_to_alias'; -SELECT id AS value, value FROM test_table; +WITH id AS value SELECT value FROM test_table; SET prefer_column_name_to_alias = 1; -SELECT id AS value, value FROM test_table; +WITH id AS value SELECT value FROM test_table; SET prefer_column_name_to_alias = 0; DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02342_analyzer_compound_types.reference b/tests/queries/0_stateless/02342_analyzer_compound_types.reference index 6d6cfb5e579..d8e8237f8f5 100644 --- a/tests/queries/0_stateless/02342_analyzer_compound_types.reference +++ b/tests/queries/0_stateless/02342_analyzer_compound_types.reference @@ -19,17 +19,17 @@ value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) value.value_1_level_0 String ('value_0_level_1','value_1_level_1') value_1_level_0 -- -value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +alias_value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) tupleElement(value, \'value_0_level_0\') Tuple(value_0_level_1 String, value_1_level_1 String) tupleElement(value, \'value_1_level_0\') String (('value_0_level_1','value_1_level_1'),'value_1_level_0') ('value_0_level_1','value_1_level_1') value_1_level_0 -- -value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +alias_value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) tupleElement(value, \'value_0_level_0\') Tuple(value_0_level_1 String, value_1_level_1 String) tupleElement(value, \'value_1_level_0\') String (('value_0_level_1','value_1_level_1'),'value_1_level_0') ('value_0_level_1','value_1_level_1') value_1_level_0 -- -value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) +alias_value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) toString(tupleElement(value, \'value_0_level_0\')) String toString(tupleElement(value, \'value_1_level_0\')) String (('value_0_level_1','value_1_level_1'),'value_1_level_0') (\'value_0_level_1\',\'value_1_level_1\') value_1_level_0 @@ -46,17 +46,17 @@ value.value_0_level_0.value_0_level_1 String value.value_0_level_0.value_1_level_1 String value_0_level_1 value_1_level_1 -- -value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(value_0_level_1 String, value_1_level_1 String) tupleElement(value.value_0_level_0, \'value_0_level_1\') String tupleElement(value.value_0_level_0, \'value_1_level_1\') String ('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 -- -value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(value_0_level_1 String, value_1_level_1 String) tupleElement(value.value_0_level_0, \'value_0_level_1\') String tupleElement(value.value_0_level_0, \'value_1_level_1\') String ('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 -- -value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value Tuple(value_0_level_1 String, value_1_level_1 String) toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String ('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 @@ -82,17 +82,17 @@ value.value_0_level_0.value_0_level_1 Array(String) value.value_0_level_0.value_1_level_1 Array(String) ['value_0_level_1'] ['value_1_level_1'] -- -value.value_0_level_0 Array(Tuple(value_0_level_1 String, value_1_level_1 String)) +alias_value Array(Tuple(value_0_level_1 String, value_1_level_1 String)) tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(String) tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(String) [('value_0_level_1','value_1_level_1')] ['value_0_level_1'] ['value_1_level_1'] -- -value.value_0_level_0 Array(Tuple(value_0_level_1 String, value_1_level_1 String)) +alias_value Array(Tuple(value_0_level_1 String, value_1_level_1 String)) tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(String) tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(String) [('value_0_level_1','value_1_level_1')] ['value_0_level_1'] ['value_1_level_1'] -- -value.value_0_level_0 Array(Tuple(value_0_level_1 String, value_1_level_1 String)) +alias_value Array(Tuple(value_0_level_1 String, value_1_level_1 String)) toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String [('value_0_level_1','value_1_level_1')] [\'value_0_level_1\'] [\'value_1_level_1\'] @@ -118,17 +118,17 @@ value.value_0_level_0.value_0_level_1 Array(Array(String)) value.value_0_level_0.value_1_level_1 Array(Array(String)) [['value_0_level_1']] [['value_1_level_1']] -- -value.value_0_level_0 Array(Nested(value_0_level_1 String, value_1_level_1 String)) +value_alias Array(Nested(value_0_level_1 String, value_1_level_1 String)) tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(Array(String)) tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(Array(String)) [[('value_0_level_1','value_1_level_1')]] [['value_0_level_1']] [['value_1_level_1']] -- -value.value_0_level_0 Array(Nested(value_0_level_1 String, value_1_level_1 String)) +value_alias Array(Nested(value_0_level_1 String, value_1_level_1 String)) tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(Array(String)) tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(Array(String)) [[('value_0_level_1','value_1_level_1')]] [['value_0_level_1']] [['value_1_level_1']] -- -value.value_0_level_0 Array(Nested(value_0_level_1 String, value_1_level_1 String)) +value_alias Array(Nested(value_0_level_1 String, value_1_level_1 String)) toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String [[('value_0_level_1','value_1_level_1')]] [[\'value_0_level_1\']] [[\'value_1_level_1\']] From 635bd44413027e1358ea65a47d058a4fc6ee2423 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 20 Jul 2022 17:25:24 +0200 Subject: [PATCH 061/188] Added support for table functions --- src/Analyzer/IQueryTreeNode.cpp | 3 +- src/Analyzer/IQueryTreeNode.h | 3 +- src/Analyzer/MatcherNode.h | 2 +- src/Analyzer/QueryAnalysisPass.cpp | 101 +++++++++++-- src/Analyzer/QueryTreeBuilder.cpp | 21 ++- src/Analyzer/TableFunctionNode.cpp | 137 ++++++++++++++++++ src/Analyzer/TableFunctionNode.h | 122 ++++++++++++++++ src/Analyzer/TableNode.cpp | 2 + src/Analyzer/TableNode.h | 2 +- .../InterpreterSelectQueryAnalyzer.cpp | 27 ++-- .../02368_analyzer_table_functions.reference | 6 + .../02368_analyzer_table_functions.sql | 10 ++ 12 files changed, 408 insertions(+), 28 deletions(-) create mode 100644 src/Analyzer/TableFunctionNode.cpp create mode 100644 src/Analyzer/TableFunctionNode.h create mode 100644 tests/queries/0_stateless/02368_analyzer_table_functions.reference create mode 100644 tests/queries/0_stateless/02368_analyzer_table_functions.sql diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 01977651180..b947d43ec0f 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -23,7 +23,7 @@ const char * toString(QueryTreeNodeType type) switch (type) { case QueryTreeNodeType::IDENTIFIER: return "IDENTIFIER"; - case QueryTreeNodeType::ASTERISK: return "ASTERISK"; + case QueryTreeNodeType::MATCHER: return "MATCHER"; case QueryTreeNodeType::TRANSFORMER: return "TRANSFORMER"; case QueryTreeNodeType::LIST: return "LIST"; case QueryTreeNodeType::CONSTANT: return "CONSTANT"; @@ -31,6 +31,7 @@ const char * toString(QueryTreeNodeType type) case QueryTreeNodeType::COLUMN: return "COLUMN"; case QueryTreeNodeType::LAMBDA: return "LAMBDA"; case QueryTreeNodeType::TABLE: return "TABLE"; + case QueryTreeNodeType::TABLE_FUNCTION: return "TABLE_FUNCTION"; case QueryTreeNodeType::QUERY: return "QUERY"; } } diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 17a7a481789..75c3af87c3a 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -28,7 +28,7 @@ class WriteBuffer; enum class QueryTreeNodeType { IDENTIFIER, - ASTERISK, + MATCHER, TRANSFORMER, LIST, CONSTANT, @@ -36,6 +36,7 @@ enum class QueryTreeNodeType COLUMN, LAMBDA, TABLE, + TABLE_FUNCTION, QUERY, }; diff --git a/src/Analyzer/MatcherNode.h b/src/Analyzer/MatcherNode.h index 00024801091..f21c759c328 100644 --- a/src/Analyzer/MatcherNode.h +++ b/src/Analyzer/MatcherNode.h @@ -138,7 +138,7 @@ public: QueryTreeNodeType getNodeType() const override { - return QueryTreeNodeType::ASTERISK; + return QueryTreeNodeType::MATCHER; } String getName() const override; diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index a34476fff5f..962923b8f54 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -36,16 +36,20 @@ #include #include #include +#include #include #include #include #include +#include + #include #include #include #include +#include #include #include @@ -63,6 +67,7 @@ namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; extern const int UNKNOWN_IDENTIFIER; + extern const int UNKNOWN_FUNCTION; extern const int LOGICAL_ERROR; extern const int CYCLIC_ALIASES; extern const int INCORRECT_RESULT_OF_SCALAR_SUBQUERY; @@ -1084,6 +1089,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL auto * table_node = from_node->as(); auto * query_node = from_node->as(); + auto * table_function_node = from_node->as(); /** Edge case scenario when subquery in FROM node try to resolve identifier from parent scopes, when FROM is not resolved. * SELECT subquery.b AS value FROM (SELECT value, 1 AS b) AS subquery; @@ -1092,8 +1098,8 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL if (query_node_from_section_in_resolve_process.contains(query_scope_node)) return {}; - if (!table_node && !query_node) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "FROM does not contain table or query node. Actual {}", from_node->formatASTForErrorMessage()); + if (!table_node && !table_function_node && !query_node) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "FROM does not contain table, table function or query node. Actual {}", from_node->formatASTForErrorMessage()); const auto & identifier = identifier_lookup.identifier; const auto & path_start = identifier.getParts().front(); @@ -1117,6 +1123,11 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL if (query_node->hasAlias()) table_expression_name = query_node->getAlias(); } + else if (table_function_node) + { + if (table_function_node->hasAlias()) + table_expression_name = table_function_node->getAlias(); + } if (identifier_lookup.isTableLookup()) { @@ -1140,9 +1151,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL { TableExpressionColumns storage_columns; - if (table_node) + if (table_node || table_function_node) { - const auto & storage_snapshot = table_node->getStorageSnapshot(); + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); const auto & columns_description = storage_snapshot->metadata->getColumns(); @@ -1540,14 +1551,18 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, NamesAndTypesList initial_matcher_columns; - if (auto * table_expression_query_node = table_expression_node->as()) + auto * table_expression_query_node = table_expression_node->as(); + auto * table_expression_table_node = table_expression_node->as(); + auto * table_expression_table_function_node = table_expression_node->as(); + + if (table_expression_query_node) { initial_matcher_columns = getQueryNodeColumns(table_expression_node); } else { - const auto & table_expression_table_node = table_expression_node->as(); - initial_matcher_columns = table_expression_table_node.getStorageSnapshot()->getColumns(GetColumnsOptions(GetColumnsOptions::All)); + const auto & storage_snapshot = table_expression_table_node ? table_expression_table_node->getStorageSnapshot() : table_expression_table_function_node->getStorageSnapshot(); + initial_matcher_columns = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All)); } for (auto & column : initial_matcher_columns) @@ -1586,13 +1601,17 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, NamesAndTypesList initial_matcher_columns; - if (auto * from_query_node = scope_query_node->getFrom()->as()) + auto * from_query_node = scope_query_node->getFrom()->as(); + auto * from_table_node = scope_query_node->getFrom()->as(); + auto * from_table_function_node = scope_query_node->getFrom()->as(); + + if (from_query_node) { initial_matcher_columns = getQueryNodeColumns(scope_query_node->getFrom()); } - else + else if (from_table_node || from_table_function_node) { - const auto & table_node = scope_query_node->getFrom()->as(); + const auto & storage_snapshot = from_table_node ? from_table_node->getStorageSnapshot() : from_table_function_node->getStorageSnapshot(); UInt8 get_column_options_kind = 0; @@ -1613,7 +1632,13 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, } auto get_columns_options = GetColumnsOptions(static_cast(get_column_options_kind)); - initial_matcher_columns = table_node.getStorageSnapshot()->getColumns(get_columns_options); + initial_matcher_columns = storage_snapshot->getColumns(get_columns_options); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unqualified matcher resolve unexpected FROM section {}", + scope_query_node->getFrom()->formatASTForErrorMessage()); } for (auto & column : initial_matcher_columns) @@ -2385,6 +2410,10 @@ void QueryAnalyzer::initializeQueryFrom(QueryTreeNodePtr & from_node, Identifier { /// Already initialized } + else if (auto * table_function = from_node->as()) + { + /// Already initialized + } else if (auto * from_table_identifier = from_node->as()) { /// TODO: Context resolve storage @@ -2478,6 +2507,53 @@ void QueryAnalyzer::resolveQueryFrom(QueryTreeNodePtr & from_node, IdentifierRes resolveQuery(from_node, subquery_scope); return; } + else if (auto * from_table_function = from_node->as()) + { + const auto & table_function_factory = TableFunctionFactory::instance(); + const auto & table_function_name = from_table_function->getTableFunctionName(); + + TableFunctionPtr table_function_ptr = table_function_factory.tryGet(table_function_name, context); + if (!table_function_ptr) + { + auto hints = TableFunctionFactory::instance().getHints(table_function_name); + if (!hints.empty()) + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}. Maybe you meant: {}", table_function_name, toString(hints)); + else + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", table_function_name); + } + + if (context->getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint()) + { + const auto & insertion_table = context->getInsertionTable(); + if (!insertion_table.empty()) + { + const auto & structure_hint = DatabaseCatalog::instance().getTable(insertion_table, context)->getInMemoryMetadataPtr()->columns; + table_function_ptr->setStructureHint(structure_hint); + } + } + + /// TODO: Special functions that can take query + /// TODO: Support qualified matchers for table function + + for (auto & argument_node : from_table_function->getArguments().getNodes()) + { + if (argument_node->getNodeType() == QueryTreeNodeType::MATCHER) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Matcher as table function argument is not supported {}. In scope {}", + from_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + + resolveExpressionNode(argument_node, scope, false /*allow_lambda_expression*/); + } + + auto table_function_ast = from_table_function->toAST(); + table_function_ptr->parseArguments(table_function_ast, context); + + auto table_function_storage = table_function_ptr->execute(table_function_ast, context, table_function_ptr->getName()); + from_table_function->resolve(std::move(table_function_ptr), std::move(table_function_storage), context); + } else if (auto * from_table = from_node->as()) { return; @@ -2546,6 +2622,9 @@ void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_node, IdentifierResolv if (query_node_typed.getFrom()) { + if (auto * table_function = query_node_typed.getFrom()->as()) + visitor.visit(table_function->getArgumentsNode()); + auto [it, _] = query_node_from_section_in_resolve_process.emplace(query_node.get()); initializeQueryFrom(query_node_typed.getFrom(), scope); diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index e134e65f309..2385c8e74d2 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include @@ -393,10 +394,26 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q return node; } - else + else if (table_expression.table_function) { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only table is supported"); + auto & table_function_expression = table_expression.table_function->as(); + + auto node = std::make_shared(table_function_expression.name); + + if (table_function_expression.arguments) + { + const auto & function_arguments_list = table_function_expression.arguments->as()->children; + for (const auto & argument : function_arguments_list) + node->getArguments().getNodes().push_back(getExpression(argument)); + } + + node->setAlias(table_function_expression.tryGetAlias()); + node->setOriginalAST(table_expression.table_function); + + return node; } + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Unsupported table expression node {}", table_element.table_expression->formatForErrorMessage()); } // if (table_element.table_join) diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp new file mode 100644 index 00000000000..7063d6ad149 --- /dev/null +++ b/src/Analyzer/TableFunctionNode.cpp @@ -0,0 +1,137 @@ +#include + +#include +#include +#include + +#include + +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +TableFunctionNode::TableFunctionNode(String table_function_name_) + : table_function_name(table_function_name_) + , storage_id("system", "one") +{ + children.resize(1); + children[arguments_child_index] = std::make_shared(); +} + +void TableFunctionNode::resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context) +{ + if (isResolved()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function node {} already resolved", table_function_name); + + table_function = std::move(table_function_value); + storage = std::move(storage_value); + storage_id = storage->getStorageID(); + storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); +} + +const StorageID & TableFunctionNode::getStorageID() const +{ + if (!storage) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function node {} is not resolved", table_function_name); + + return storage_id; +} + +const StorageSnapshotPtr & TableFunctionNode::getStorageSnapshot() const +{ + if (!storage) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function node {} is not resolved", table_function_name); + + return storage_snapshot; +} + +String TableFunctionNode::getName() const +{ + String name = table_function_name; + + const auto & arguments = getArguments(); + name += '('; + name += arguments.getName(); + name += ')'; + + return name; +} + +void TableFunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const +{ + buffer << std::string(indent, ' ') << "TABLE_FUNCTION id: " << format_state.getNodeId(this); + + if (hasAlias()) + buffer << ", alias: " << getAlias(); + + buffer << ", table_function_name: " << table_function_name; + + const auto & arguments = getArguments(); + if (!arguments.getNodes().empty()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "ARGUMENTS\n"; + arguments.dumpTreeImpl(buffer, format_state, indent + 4); + } +} + +bool TableFunctionNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + if (table_function_name != rhs_typed.table_function_name) + return false; + + if (storage && rhs_typed.storage) + return storage_id == rhs_typed.storage_id; + + return true; +} + +void TableFunctionNode::updateTreeHashImpl(HashState & state) const +{ + state.update(table_function_name.size()); + state.update(table_function_name); + + if (storage) + { + auto full_name = storage_id.getFullNameNotQuoted(); + state.update(full_name.size()); + state.update(full_name); + } +} + +ASTPtr TableFunctionNode::toASTImpl() const +{ + auto table_function_ast = std::make_shared(); + + table_function_ast->name = table_function_name; + + const auto & arguments = getArguments(); + if (!arguments.getNodes().empty()) + { + table_function_ast->children.push_back(arguments.toAST()); + table_function_ast->arguments = table_function_ast->children.back(); + } + + return table_function_ast; +} + +QueryTreeNodePtr TableFunctionNode::cloneImpl() const +{ + auto result = std::make_shared(table_function_name); + + result->storage = storage; + result->storage_id = storage_id; + result->storage_snapshot = storage_snapshot; + + return result; +} + +} diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h new file mode 100644 index 00000000000..f664812e6b2 --- /dev/null +++ b/src/Analyzer/TableFunctionNode.h @@ -0,0 +1,122 @@ +#pragma once + +#include +#include +#include + +#include +#include + +#include +#include + + +namespace DB +{ + +class ITableFunction; +using TableFunctionPtr = std::shared_ptr; + +/** Table function node represents table function in query tree. + * Example: SELECT a FROM table_function(arguments...). + * + * In query tree table function arguments are represented by ListNode. + * + * Table function resolution must be done during query analysis pass. + */ +class TableFunctionNode; +using TableFunctionNodePtr = std::shared_ptr; + +class TableFunctionNode : public IQueryTreeNode +{ +public: + /// Construct table function node with table function name + explicit TableFunctionNode(String table_function_name); + + /// Get table function name + const String & getTableFunctionName() const + { + return table_function_name; + } + + /// Get arguments + const ListNode & getArguments() const + { + return children[arguments_child_index]->as(); + } + + /// Get arguments + ListNode & getArguments() + { + return children[arguments_child_index]->as(); + } + + /// Get arguments node + const QueryTreeNodePtr & getArgumentsNode() const + { + return children[arguments_child_index]; + } + + /// Get arguments node + QueryTreeNodePtr & getArgumentsNode() + { + return children[arguments_child_index]; + } + + /// Returns true, if table function is resolved, false otherwise + bool isResolved() const + { + return storage != nullptr && table_function != nullptr; + } + + /// Get table function, returns nullptr if table function node is not resolved + const TableFunctionPtr & getTableFunction() const + { + return table_function; + } + + /// Get storage, returns nullptr if table function node is not resolved + const StoragePtr & getStorage() const + { + return storage; + } + + /// Resolve table function with table_function, storage and context + void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context); + + /// Get storage id, throws exception if function node is not resolved + const StorageID & getStorageID() const; + + /// Get storage snapshot, throws exception if function node is not resolved + const StorageSnapshotPtr & getStorageSnapshot() const; + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::TABLE_FUNCTION; + } + + String getName() const override; + + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + + void updateTreeHashImpl(HashState & state) const override; + +protected: + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + static constexpr size_t arguments_child_index = 0; + + String table_function_name; + TableFunctionPtr table_function; + StoragePtr storage; + StorageID storage_id; + StorageSnapshotPtr storage_snapshot; +}; + +} + diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index 65dbccfd27a..cc5de07b916 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -21,6 +21,8 @@ TableNode::TableNode(StoragePtr storage_, ContextPtr context) { } +TableNode::TableNode() : storage_id("system", "one") {} + void TableNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "TABLE id: " << format_state.getNodeId(this); diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index 25f6529d489..6756381ed34 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -80,7 +80,7 @@ protected: QueryTreeNodePtr cloneImpl() const override; private: - TableNode() : storage_id("", "") {} + TableNode(); StoragePtr storage; StorageID storage_id; diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 63768d4f16e..7bdc3968d7c 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -16,8 +16,9 @@ #include #include #include -#include #include +#include +#include #include #include #include @@ -424,8 +425,7 @@ public: if (!column_node) return; - auto * table_node = column_node->getColumnSource()->as(); - if (!table_node) + if (column_node->getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA) return; /// Replace ALIAS column with expression @@ -606,31 +606,36 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() query_info.original_query = query; query_info.query = query; - if (auto * table_node = query_tree_typed.getFrom()->as()) + auto * table_node = query_tree_typed.getFrom()->as(); + auto * table_function_node = query_tree_typed.getFrom()->as(); + auto * query_node = query_tree_typed.getFrom()->as(); + + if (table_node || table_function_node) { - auto from_stage = table_node->getStorage()->getQueryProcessingStage( - current_context, select_query_options.to_stage, table_node->getStorageSnapshot(), query_info); + const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + + auto from_stage = storage->getQueryProcessingStage(current_context, select_query_options.to_stage, storage_snapshot, query_info); Names column_names(source_columns_set.begin(), source_columns_set.end()); - if (column_names.empty() && table_node->getStorage()->getName() == "SystemOne") + if (column_names.empty() && storage->getName() == "SystemOne") column_names.push_back("dummy"); if (!column_names.empty()) - table_node->getStorage()->read( - query_plan, column_names, table_node->getStorageSnapshot(), query_info, getContext(), from_stage, max_block_size, max_streams); + storage->read(query_plan, column_names, storage_snapshot, query_info, getContext(), from_stage, max_block_size, max_streams); /// Create step which reads from empty source if storage has no data. if (!query_plan.isInitialized()) { - auto source_header = table_node->getStorageSnapshot()->getSampleBlockForColumns(column_names); + auto source_header = storage_snapshot->getSampleBlockForColumns(column_names); Pipe pipe(std::make_shared(source_header)); auto read_from_pipe = std::make_unique(std::move(pipe)); read_from_pipe->setStepDescription("Read from NullSource"); query_plan.addStep(std::move(read_from_pipe)); } } - else if (auto * query_node = query_tree_typed.getFrom()->as()) + else if (query_node) { InterpreterSelectQueryAnalyzer interpeter(query_tree_typed.getFrom(), select_query_options, getContext()); interpeter.initializeQueryPlanIfNeeded(); diff --git a/tests/queries/0_stateless/02368_analyzer_table_functions.reference b/tests/queries/0_stateless/02368_analyzer_table_functions.reference new file mode 100644 index 00000000000..0c1bc4d90a2 --- /dev/null +++ b/tests/queries/0_stateless/02368_analyzer_table_functions.reference @@ -0,0 +1,6 @@ +1 2 [1,2,3] [['abc'],[],['d','e']] +1 2 [1,2,3] [['abc'],[],['d','e']] +1 2 [1,2,3] [['abc'],[],['d','e']] +1 2 [1,2,3] [['abc'],[],['d','e']] +1 2 [1,2,3] [['abc'],[],['d','e']] +CSV 1,2,"[1,2,3]","[[\'abc\'], [], [\'d\', \'e\']]" 1 2 [1,2,3] [['abc'],[],['d','e']] diff --git a/tests/queries/0_stateless/02368_analyzer_table_functions.sql b/tests/queries/0_stateless/02368_analyzer_table_functions.sql new file mode 100644 index 00000000000..6d9accef30c --- /dev/null +++ b/tests/queries/0_stateless/02368_analyzer_table_functions.sql @@ -0,0 +1,10 @@ +SET use_analyzer = 1; + +SELECT c1, c2, c3, c4 FROM format('CSV', '1,2,"[1,2,3]","[[\'abc\'], [], [\'d\', \'e\']]"'); +SELECT f.c1, f.c2, f.c3, f.c4 FROM format('CSV', '1,2,"[1,2,3]","[[\'abc\'], [], [\'d\', \'e\']]"') AS f; +SELECT f.* FROM format('CSV', '1,2,"[1,2,3]","[[\'abc\'], [], [\'d\', \'e\']]"') AS f; + +WITH 'CSV', '1,2,"[1,2,3]","[[\'abc\'], [], [\'d\', \'e\']]"' AS format_value SELECT c1, c2, c3, c4 FROM format('CSV', format_value); +WITH concat('1,2,"[1,2,3]",','"[[\'abc\'], [], [\'d\', \'e\']]"') AS format_value SELECT c1, c2, c3, c4 FROM format('CSV', format_value); + +SELECT format, format_value, c1, c2, c3, c4 FROM format('CSV' AS format, '1,2,"[1,2,3]","[[\'abc\'], [], [\'d\', \'e\']]"' AS format_value); From a9e1f74696f22751f70ed59d1b7663db53a9cce6 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 21 Jul 2022 13:02:41 +0200 Subject: [PATCH 062/188] Added support for grouping function --- src/Analyzer/QueryAnalysisPass.cpp | 16 +++++++++++----- src/Analyzer/QueryAnalysisPass.h | 6 +++++- .../InterpreterSelectQueryAnalyzer.cpp | 14 ++++++++++++++ 3 files changed, 30 insertions(+), 6 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 962923b8f54..8d9de11f5bf 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -78,7 +78,7 @@ namespace ErrorCodes } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. - * And additional documentation is writter for each method, where special cases are described in detail. + * And additional documentation for each method, where special cases are described in detail. * * Each node in query must be resolved. For each query tree node resolved state is specific. * @@ -86,7 +86,7 @@ namespace ErrorCodes * * For table node no resolve process exists, it is resolved during construction. * - * For function node to be resolved parameters and arguments must be resolved, must be initialized with concrete aggregate or + * For function node to be resolved parameters and arguments must be resolved, function node must be initialized with concrete aggregate or * non aggregate function and with result type. * * For lambda node there can be 2 different cases. @@ -111,7 +111,7 @@ namespace ErrorCodes * * If there are no information of identifier context rules are following: * 1. Try to resolve identifier in expression context. - * 2. Try to resolve identifier in function context, if it is allowed. Example: SELECT func; Here func identifier cannot be resolved in function context + * 2. Try to resolve identifier in function context, if it is allowed. Example: SELECT func(arguments); Here func identifier cannot be resolved in function context * because query projection does not support that. * 3. Try to resolve identifier in talbe context, if it is allowed. Example: SELECT table; Here table identifier cannot be resolved in function context * because query projection does not support that. @@ -124,8 +124,8 @@ namespace ErrorCodes * 2. Try to resolve identifier from aliases. * 3. Try to resolve identifier from tables if scope is query, or if there are registered table columns in scope. * Steps 2 and 3 can be changed using prefer_column_name_to_alias setting. - * If identifier could not be resolved in current scope, resolution must be continued in parent scopes. * 4. If it is table lookup, try to resolve identifier from CTE. + * If identifier could not be resolved in current scope, resolution must be continued in parent scopes. * 5. Try to resolve identifier from parent scopes. * * Additional rules about aliases and scopes. @@ -159,9 +159,10 @@ namespace ErrorCodes * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. + * TODO: UNION * TODO: JOIN, ARRAY JOIN * TODO: bulding sets - * TODO: Special functions grouping, in. + * TODO: Special functions in. */ /// Identifier lookup context @@ -2075,6 +2076,11 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc node = result_list; return; } + else if (function_name == "grouping") + { + /// It is responsibility of planner to handle grouping function + return; + } /** Try to resolve function as * 1. Executable user defined function. diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index 8dc8e255e53..78da924e96f 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -26,7 +26,11 @@ namespace DB * TODO: Scalar subqueries must be evaluated only if they are part of query tree where we must have constant. This is currently not done * because execution layer does not support scalar subqueries execution. * - * 8. Special function `untuple` is handled properly. + * 8. Special functions handling: + * Function `untuple` is handled properly. + * Function `arrayJoin` is handled properly. + * For function `grouping` arguments are resolved, but it is planner responsibility to initialize it with concrete grouping function + * based on group by kind and group by keys positions. * 9. Sets are build for `in` function. */ class QueryAnalysisPass final : public IQueryTreePass diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 7bdc3968d7c..6e33dff7363 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -45,6 +45,8 @@ namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; extern const int LOGICAL_ERROR; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } QueryPipeline buildDummyPipeline() @@ -353,6 +355,18 @@ private: { auto function_node_name = function_node.getName(); + if (function_node.getFunctionName() == "grouping") + { + size_t arguments_size = function_node.getArguments().getNodes().size(); + + if (arguments_size == 0) + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function GROUPING expects at least one argument"); + else if (arguments_size > 64) + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Function GROUPING can have up to 64 arguments, but {} provided", arguments_size); + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function GROUPING is not supported"); + } + if (function_node.isAggregateFunction()) { size_t actions_stack_size = actions_stack.size(); From 2148d0e58d7a67d632b2aaab28ecedf20f91bb28 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 21 Jul 2022 16:45:32 +0200 Subject: [PATCH 063/188] Added support for arrayJoin function --- .../InterpreterSelectQueryAnalyzer.cpp | 40 ++++++++- ...369_analyzer_array_join_function.reference | 82 +++++++++++++++++++ .../02369_analyzer_array_join_function.sql | 60 ++++++++++++++ 3 files changed, 180 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02369_analyzer_array_join_function.reference create mode 100644 tests/queries/0_stateless/02369_analyzer_array_join_function.sql diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 6e33dff7363..5fe7b9c1f20 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -134,6 +134,18 @@ struct QueryTreeActionsScopeNode return node; } + const ActionsDAG::Node * addInputConstantColumnIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) + { + auto it = node_name_to_node.find(node_name); + if (it != node_name_to_node.end()) + return it->second; + + const auto * node = &actions_dag->addInput(column); + node_name_to_node[node->result_name] = node; + + return node; + } + const ActionsDAG::Node * addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) { auto it = node_name_to_node.find(node_name); @@ -166,6 +178,18 @@ struct QueryTreeActionsScopeNode return node; } + const ActionsDAG::Node * addArrayJoinIfNecessary(const std::string & node_name, const ActionsDAG::Node * child) + { + auto it = node_name_to_node.find(node_name); + if (it != node_name_to_node.end()) + return it->second; + + const auto * node = &actions_dag->addArrayJoin(*child, node_name); + node_name_to_node[node->result_name] = node; + + return node; + } + std::unordered_map node_name_to_node; ActionsDAGPtr actions_dag; QueryTreeNodePtr scope_node; @@ -253,7 +277,7 @@ private: for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputColumnIfNecessary(constant_name, column.type); + actions_stack_node.addInputConstantColumnIfNecessary(constant_name, column); } return {constant_name, 0}; @@ -408,7 +432,19 @@ private: for (auto & function_argument_node_name : function_arguments_node_names) children.push_back(actions_stack[level].getNodeOrThrow(function_argument_node_name)); - actions_stack[level].addFunctionIfNecessary(function_node_name, children, function_node.getFunction()); + if (function_node.getFunctionName() == "arrayJoin") + { + if (level != 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Expression in arrayJoin cannot depend on lambda argument: {} ", + function_arguments_node_names.at(0)); + + actions_stack[level].addArrayJoinIfNecessary(function_node_name, children.at(0)); + } + else + { + actions_stack[level].addFunctionIfNecessary(function_node_name, children, function_node.getFunction()); + } size_t actions_stack_size = actions_stack.size(); for (size_t i = level + 1; i < actions_stack_size; ++i) diff --git a/tests/queries/0_stateless/02369_analyzer_array_join_function.reference b/tests/queries/0_stateless/02369_analyzer_array_join_function.reference new file mode 100644 index 00000000000..8fa16a27152 --- /dev/null +++ b/tests/queries/0_stateless/02369_analyzer_array_join_function.reference @@ -0,0 +1,82 @@ +1 +2 +3 +-- +1 1 +2 2 +3 3 +-- +1 1 +2 2 +3 3 +-- +[1,2,3] 1 +[1,2,3] 2 +[1,2,3] 3 +-- +1 1 +1 2 +1 3 +1 4 +2 1 +2 2 +2 3 +2 4 +3 1 +3 2 +3 3 +3 4 +-- +[1,1,1] +[2,2,2] +[3,3,3] +-- +[2,3,4] 1 +[3,4,5] 2 +[4,5,6] 3 +-- +0 1 +0 2 +0 3 +-- +0 1 1 +0 2 2 +0 3 3 +-- +0 1 1 +0 1 2 +0 1 3 +0 1 4 +0 2 1 +0 2 2 +0 2 3 +0 2 4 +0 3 1 +0 3 2 +0 3 3 +0 3 4 +-- +0 1 1 5 +0 1 1 6 +0 1 2 5 +0 1 2 6 +0 1 3 5 +0 1 3 6 +0 1 4 5 +0 1 4 6 +0 2 1 5 +0 2 1 6 +0 2 2 5 +0 2 2 6 +0 2 3 5 +0 2 3 6 +0 2 4 5 +0 2 4 6 +0 3 1 5 +0 3 1 6 +0 3 2 5 +0 3 2 6 +0 3 3 5 +0 3 3 6 +0 3 4 5 +0 3 4 6 diff --git a/tests/queries/0_stateless/02369_analyzer_array_join_function.sql b/tests/queries/0_stateless/02369_analyzer_array_join_function.sql new file mode 100644 index 00000000000..08b3bc3c146 --- /dev/null +++ b/tests/queries/0_stateless/02369_analyzer_array_join_function.sql @@ -0,0 +1,60 @@ +SET use_analyzer = 1; + + +SELECT arrayJoin([1, 2, 3]); + +SELECT '--'; + +SELECT arrayJoin([1, 2, 3]) AS a, arrayJoin([1, 2, 3]); + +SELECT '--'; + +SELECT arrayJoin([1, 2, 3]) AS a, a; + +SELECT '--'; + +SELECT arrayJoin([[1, 2, 3]]) AS a, arrayJoin(a) AS b; + +SELECT '--'; + +SELECT arrayJoin([1, 2, 3]) AS a, arrayJoin([1, 2, 3, 4]) AS b; + +SELECT '--'; + +SELECT arrayMap(x -> arrayJoin([1, 2, 3]), [1, 2, 3]); + +SELECT arrayMap(x -> arrayJoin(x), [[1, 2, 3]]); -- { serverError 36 } + +SELECT arrayMap(x -> arrayJoin(cast(x, 'Array(UInt8)')), [[1, 2, 3]]); -- { serverError 36 } + +SELECT '--'; + +SELECT arrayMap(x -> x + a, [1, 2, 3]), arrayJoin([1,2,3]) as a; + +SELECT '--'; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value_1 Array(UInt8), + value_2 Array(UInt8), +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, [1, 2, 3], [1, 2, 3, 4]); + +SELECT id, arrayJoin(value_1) FROM test_table; + +SELECT '--'; + +SELECT id, arrayJoin(value_1) AS a, a FROM test_table; + +SELECT '--'; + +SELECT id, arrayJoin(value_1), arrayJoin(value_2) FROM test_table; + +SELECT '--'; + +SELECT id, arrayJoin(value_1), arrayJoin(value_2), arrayJoin([5, 6]) FROM test_table; + +DROP TABLE test_table; From 062e17052f11ae5acf0a16bd512363dbb9b60c9c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 22 Jul 2022 17:09:56 +0200 Subject: [PATCH 064/188] Added support for IN function --- src/Analyzer/QueryAnalysisPass.cpp | 785 +++++++++++------- src/Analyzer/QueryAnalysisPass.h | 3 +- src/Analyzer/QueryNode.cpp | 33 + src/Analyzer/QueryNode.h | 4 +- src/Analyzer/SetUtils.cpp | 183 ++++ src/Analyzer/SetUtils.h | 30 + src/Analyzer/Utils.cpp | 39 + src/Analyzer/Utils.h | 14 + .../InterpreterSelectQueryAnalyzer.cpp | 5 + .../02370_analyzer_in_function.reference | 14 + .../02370_analyzer_in_function.sql | 23 + 11 files changed, 844 insertions(+), 289 deletions(-) create mode 100644 src/Analyzer/SetUtils.cpp create mode 100644 src/Analyzer/SetUtils.h create mode 100644 src/Analyzer/Utils.cpp create mode 100644 src/Analyzer/Utils.h create mode 100644 tests/queries/0_stateless/02370_analyzer_in_function.reference create mode 100644 tests/queries/0_stateless/02370_analyzer_in_function.sql diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 8d9de11f5bf..c5dc498eba5 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -1,7 +1,5 @@ #include -#include - #include #include @@ -25,9 +23,15 @@ #include #include #include +#include +#include #include +#include +#include +#include +#include #include #include #include @@ -45,6 +49,7 @@ #include +#include #include #include #include @@ -52,6 +57,7 @@ #include #include #include +#include #include @@ -75,6 +81,7 @@ namespace ErrorCodes extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS; extern const int UNKNOWN_DATABASE; extern const int UNKNOWN_TABLE; + extern const int INCORRECT_ELEMENT_OF_SET; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -155,14 +162,14 @@ namespace ErrorCodes * lookup should not be continued, and exception must be throwed because if lookup continues identifier can be resolved from parent scope. * * TODO: Add expression name into query tree node. Example: SELECT plus(1, 1). Result: SELECT 2. Expression name of constant node should be 2. + * TODO: Update exception messages * TODO: Table identifiers with optional UUID. * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. * TODO: UNION * TODO: JOIN, ARRAY JOIN - * TODO: bulding sets - * TODO: Special functions in. + * TODO: WINDOW functions */ /// Identifier lookup context @@ -240,6 +247,93 @@ struct IdentifierLookupHash } }; +enum class IdentifierResolvePlace : UInt8 +{ + NONE = 0, + EXPRESSION_ARGUMENTS, + ALIASES, + TABLES, + /// Valid only for table lookup + CTE, + /// Valid only for table lookup + DATABASE_CATALOG +}; + +static const char * toString(IdentifierResolvePlace resolved_identifier_place) +{ + switch (resolved_identifier_place) + { + case IdentifierResolvePlace::NONE: return "NONE"; + case IdentifierResolvePlace::EXPRESSION_ARGUMENTS: return "EXPRESSION_ARGUMENTS"; + case IdentifierResolvePlace::ALIASES: return "ALIASES"; + case IdentifierResolvePlace::TABLES: return "TABLES"; + case IdentifierResolvePlace::CTE: return "CTE"; + case IdentifierResolvePlace::DATABASE_CATALOG: return "DATABASE_CATALOG"; + } +} + +struct IdentifierResolveResult +{ + IdentifierResolveResult() = default; + + QueryTreeNodePtr resolved_identifier; + IdentifierResolvePlace resolve_place = IdentifierResolvePlace::NONE; + bool resolved_from_parent_scopes = false; + + bool isResolved() const + { + return resolve_place != IdentifierResolvePlace::NONE; + } + + bool isResolvedFromParentScopes() const + { + return resolved_from_parent_scopes; + } + + bool isResolvedFromExpressionArguments() const + { + return resolve_place == IdentifierResolvePlace::EXPRESSION_ARGUMENTS; + } + + bool isResolvedFromAliases() const + { + return resolve_place == IdentifierResolvePlace::ALIASES; + } + + bool isResolvedFromTables() const + { + return resolve_place == IdentifierResolvePlace::TABLES; + } + + bool isResolvedFromCTEs() const + { + return resolve_place == IdentifierResolvePlace::CTE; + } + + void dump(WriteBuffer & buffer) const + { + if (!resolved_identifier) + { + buffer << "unresolved"; + return; + } + + buffer << resolved_identifier->formatASTForErrorMessage() << " place " << toString(resolve_place) << " resolved from parent scopes " << resolved_from_parent_scopes; + } +}; + +struct IdentifierResolveSettings +{ + /// Allow to check parent scopes during identifier resolution + bool allow_to_check_parent_scopes = true; + + /// Allow to check CTEs during table identifer resolution + bool allow_to_check_cte = true; + + /// Allow to check database catalog during table identifier resolution + bool allow_to_check_database_catalog = true; +}; + struct TableExpressionColumns { std::unordered_map column_name_to_column_node; @@ -355,7 +449,7 @@ struct IdentifierResolveScope QueryTreeNodePtr scope_node; IdentifierResolveScope * parent_scope = nullptr; - std::unordered_map identifier_lookup_to_node; + std::unordered_map identifier_lookup_to_result; /// Lambda argument can be expression like constant, column, or it can be function std::unordered_map expression_argument_name_to_node; @@ -375,7 +469,7 @@ struct IdentifierResolveScope /// CTE name to query node std::unordered_map cte_name_to_query_node; - /// Nodes with duplicated identifiers + /// Nodes with duplicated aliases std::unordered_set nodes_with_duplicated_aliases; /// Current scope expression in resolve process stack @@ -391,13 +485,11 @@ struct IdentifierResolveScope void dump(WriteBuffer & buffer) { buffer << "Scope node " << scope_node->formatASTForErrorMessage() << '\n'; - buffer << "Identifier lookup to node " << identifier_lookup_to_node.size() << '\n'; - for (const auto & [identifier, node] : identifier_lookup_to_node) + buffer << "Identifier lookup to result " << identifier_lookup_to_result.size() << '\n'; + for (const auto & [identifier, result] : identifier_lookup_to_result) { - if (!node) - buffer << "Identifier " << identifier.dump() << " in resolve process\n"; - else - buffer << "Identifier " << identifier.dump() << " node type " << node->getNodeTypeName() << " format " << node->formatASTForErrorMessage() << '\n'; + buffer << "Identifier " << identifier.dump() << " resolve result "; + result.dump(buffer); } buffer << "Expression argument name to node " << expression_argument_name_to_node.size() << '\n'; @@ -449,11 +541,11 @@ public: } else if (node->getNodeType() == QueryTreeNodeType::LIST) { - resolveExpressionNodeList(node, scope, false /*allow_lambda_expression*/); + resolveExpressionNodeList(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } else if (node->getNodeType() == QueryTreeNodeType::FUNCTION) { - resolveExpressionNode(node, scope, false /*allow_lambda_expression*/); + resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } else if (node->getNodeType() == QueryTreeNodeType::LAMBDA) { @@ -471,8 +563,6 @@ public: private: /// Utility functions - static bool isNodePartOfSubtree(const IQueryTreeNode * node, const IQueryTreeNode * root); - static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path); static NamesAndTypesList getQueryNodeColumns(const QueryTreeNodePtr & query_node); @@ -483,19 +573,19 @@ private: /// Resolve identifier functions + QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier); + QueryTreeNodePtr tryResolveIdentifierFromExpressionArguments(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); - QueryTreeNodePtr tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, bool allow_to_check_parent_scopes = true); + QueryTreeNodePtr tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, IdentifierResolveSettings identifier_resolve_settings = {}); QueryTreeNodePtr tryResolveIdentifierFromTableColumns(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); QueryTreeNodePtr tryResolveIdentifierFromTables(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); - // QueryTreeNodePtr tryResolveIdentifierInCurrentScope(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & current_scope); + IdentifierResolveResult tryResolveIdentifierInParentScopes(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); - QueryTreeNodePtr tryResolveIdentifierInParentScopes(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); - - QueryTreeNodePtr tryResolveIdentifier(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, bool allow_to_check_parent_scopes = true); + IdentifierResolveResult tryResolveIdentifier(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, IdentifierResolveSettings identifier_resolve_settings = {}); /// Resolve query tree nodes functions @@ -505,9 +595,9 @@ private: void resolveFunction(QueryTreeNodePtr & function_node, IdentifierResolveScope & scope); - void resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression); + void resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); - void resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression); + void resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); void initializeQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope); @@ -530,29 +620,6 @@ private: /// Utility functions implementation -bool QueryAnalyzer::isNodePartOfSubtree(const IQueryTreeNode * node, const IQueryTreeNode * root) -{ - std::vector nodes_to_process; - nodes_to_process.push_back(root); - - while (!nodes_to_process.empty()) - { - const auto * subtree_node = nodes_to_process.back(); - nodes_to_process.pop_back(); - - if (subtree_node == node) - return true; - - for (const auto & child : subtree_node->getChildren()) - { - if (child) - nodes_to_process.push_back(child.get()); - } - } - - return false; -} - /** Wrap expression node in tuple element function calls for nested paths. * Example: Expression node: compound_expression. Nested path: nested_path_1.nested_path_2. * Result: tupleElement(tupleElement(compound_expression, 'nested_path_1'), 'nested_path_2'). @@ -719,6 +786,46 @@ void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node) /// Resolve identifier functions implementation +/// Try resolve table identifier from database catalog +QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier) +{ + size_t parts_size = table_identifier.getPartsSize(); + if (parts_size < 1 || parts_size > 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Table identifier should consist of 1 or 2 parts. Actual {}", + table_identifier.getFullName()); + + std::string database_name; + std::string table_name; + + if (table_identifier.isCompound()) + { + database_name = table_identifier[0]; + table_name = table_identifier[1]; + } + else + { + table_name = table_identifier[0]; + } + + if (database_name.empty()) + database_name = context->getCurrentDatabase(); + + /// TODO: Context resolve storage + + auto & database_catalog = DatabaseCatalog::instance(); + auto database = database_catalog.tryGetDatabase(database_name); + if (!database) + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exists", database_name); + + auto table = database->tryGetTable(table_name, context); + + if (!table) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exists", table_name); + + return std::make_shared(std::move(table), context); +} + /** Resolve identifier from expression arguments. * * Expression arguments can be initialized during lambda analysis or they could be provided externally. @@ -771,8 +878,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(cons /** Visitor that extracts expression and function aliases from node and initialize scope tables with it. * Does not go into child lambdas and queries. * - * TODO: Maybe better for this visitor to handle QueryNode. Handle table nodes. - * * Important: * Identifier nodes with aliases are added both in alias to expression and alias to function map. * @@ -783,10 +888,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(cons * In this example id as value is identifier node that has alias, during scope initialization we cannot derive * that id is actually lambda or expression. * - * - * TODO: Disable identifier with alias node propagation for table nodes. This can occur only for special functions - * if their argument can be table. - * * There are no easy solution here, without trying to make full featured expression resolution at this stage. * Example: * WITH (x -> x + 1) AS id, id AS id_1, id_1 AS id_2 SELECT id_2(1); @@ -912,7 +1013,7 @@ using ScopeAliasVisitor = ScopeAliasVisitorMatcher::Visitor; * * 8. If identifier lookup is in expression context, clone result expression. */ -QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, bool allow_to_check_parent_scopes) +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, IdentifierResolveSettings identifier_resolve_settings) { const auto & identifier_bind_part = identifier_lookup.identifier.front(); @@ -943,13 +1044,11 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier const auto * root_expression = scope.expressions_in_resolve_process_stack.getRoot(); const auto * top_expression = scope.expressions_in_resolve_process_stack.getTop(); - if (!isNodePartOfSubtree(top_expression, root_expression)) - { + if (!isNodePartOfTree(top_expression, root_expression)) throw Exception(ErrorCodes::CYCLIC_ALIASES, "Cyclic aliases for identifier {}. In scope {}", identifier_lookup.identifier.getFullName(), scope.scope_node->formatASTForErrorMessage()); - } scope.non_cached_identifier_lookups_during_expression_resolve.insert(identifier_lookup); return {}; @@ -960,7 +1059,8 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier /// Resolve expression if necessary if (auto * alias_identifier = it->second->as()) { - it->second = tryResolveIdentifier(IdentifierLookup{alias_identifier->getIdentifier(), identifier_lookup.lookup_context}, scope, allow_to_check_parent_scopes); + auto lookup_result = tryResolveIdentifier(IdentifierLookup{alias_identifier->getIdentifier(), identifier_lookup.lookup_context}, scope, identifier_resolve_settings); + it->second = lookup_result.resolved_identifier; if (!it->second) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, @@ -1042,7 +1142,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableColumns(const Ident } if (it->second->hasAliasExpression()) - resolveExpressionNode(it->second->getAliasExpression(), scope, false /*allow_lambda_node*/); + resolveExpressionNode(it->second->getAliasExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); QueryTreeNodePtr result = it->second; @@ -1198,7 +1298,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL visitor.visit(alias_column_to_resolve->getAliasExpression()); - resolveExpressionNode(alias_column_resolve_scope.scope_node, alias_column_resolve_scope, false /*allow_lambda_expression*/); + resolveExpressionNode(alias_column_resolve_scope.scope_node, alias_column_resolve_scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); column_name_to_column_node = std::move(alias_column_resolve_scope.column_name_to_column_node); column_name_to_column_node[alias_column_to_resolve_name] = alias_column_to_resolve; @@ -1309,47 +1409,14 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL return {}; } -/** Resolve identifier in current scope. - * 1. Try resolve identifier from expression arguments. - * If prefer_column_name_to_alias = true. - * 2. Try to resolve identifier from tables. - * 3. Try to resolve identifier from aliases. - * Otherwise. - * 2. Try to resolve identifier from aliases. - * 3. Try to resolve identifier from tables. - */ -// QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierInCurrentScope(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & current_scope) -// { -// auto resolved_identifier = tryResolveIdentifierFromExpressionArguments(identifier_lookup, current_scope); -// if (resolved_identifier) -// return resolved_identifier; - -// bool prefer_column_name_to_alias = context->getSettingsRef().prefer_column_name_to_alias; -// if (unlikely(prefer_column_name_to_alias)) -// { -// resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, current_scope); -// if (resolved_identifier) -// return resolved_identifier; - -// return tryResolveIdentifierFromAliases(identifier_lookup, current_scope); -// } - -// resolved_identifier = tryResolveIdentifierFromAliases(identifier_lookup, current_scope); -// if (resolved_identifier) -// return resolved_identifier; - -// return tryResolveIdentifierFromTables(identifier_lookup, current_scope); -// } - /** Try resolve identifier in current scope parent scopes. - * If initial scope is query. Then return nullptr. - * TODO: CTE, constants can be used from parent query with statement. + * * TODO: If column is matched, throw exception that nested subqueries are not supported. * * If initial scope is expression. Then try to resolve identifier in parent scopes until query scope is hit. * For query scope resolve strategy is same as if initial scope if query. */ -QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierInParentScopes(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) { if (!scope.allow_to_check_parent_scopes) return {}; @@ -1357,15 +1424,19 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierInParentScopes(const Identif bool initial_scope_is_query = scope.scope_node->getNodeType() == QueryTreeNodeType::QUERY; bool initial_scope_is_expression = !initial_scope_is_query; + IdentifierResolveSettings identifier_resolve_settings; + identifier_resolve_settings.allow_to_check_parent_scopes = false; + identifier_resolve_settings.allow_to_check_database_catalog = false; + IdentifierResolveScope * scope_to_check = scope.parent_scope; if (initial_scope_is_expression) { while (scope_to_check != nullptr) { - auto resolved_identifier = tryResolveIdentifier(identifier_lookup, *scope_to_check, false /*allow_to_check_parent_scopes*/); - if (resolved_identifier) - return resolved_identifier; + auto resolve_result = tryResolveIdentifier(identifier_lookup, *scope_to_check, identifier_resolve_settings); + if (resolve_result.resolved_identifier) + return resolve_result; bool scope_was_query = scope_to_check->scope_node->getNodeType() == QueryTreeNodeType::QUERY; scope_to_check = scope_to_check->parent_scope; @@ -1377,27 +1448,44 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierInParentScopes(const Identif while (scope_to_check != nullptr) { - auto resolved_identifier = tryResolveIdentifier(identifier_lookup, *scope_to_check, false /*allow_to_check_parent_scopes*/); - if (resolved_identifier) - { - if (resolved_identifier->as()) - return resolved_identifier; - else if (resolved_identifier->as() && resolved_identifier->as()->isCTE()) - return resolved_identifier; - - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Resolve identifier {} from parent scope only supported for constants and CTE. In scope {}", - identifier_lookup.identifier.getFullName(), - scope.scope_node->formatASTForErrorMessage()); - } + auto lookup_result = tryResolveIdentifier(identifier_lookup, *scope_to_check, identifier_resolve_settings); + const auto & resolved_identifier = lookup_result.resolved_identifier; scope_to_check = scope_to_check->parent_scope; + + if (resolved_identifier) + { + bool is_cte = resolved_identifier->as() && resolved_identifier->as()->isCTE(); + + /** From parent scopes we can resolve table identifiers only as CTE. + * Example: SELECT (SELECT 1 FROM a) FROM test_table AS a; + * + * During child scope table identifier resolve a, table node test_table with alias a from parent scope + * is invalid. + */ + if (identifier_lookup.isTableLookup() && !is_cte) + continue; + + if (resolved_identifier->as()) + return lookup_result; + else if (is_cte) + return lookup_result; + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Resolve identifier {} from parent scope only supported for constants and CTE. Actual {} node type {}. In scope {}", + identifier_lookup.identifier.getFullName(), + resolved_identifier->formatASTForErrorMessage(), + resolved_identifier->getNodeTypeName(), + scope.scope_node->formatASTForErrorMessage()); + } } return {}; } /** Resolve identifier in scope. + * + * If identifier was resolved resolve identified lookup status will be updated. * * Steps: * 1. Register identifier lookup in scope identifier_lookup_to_resolve_status table. @@ -1405,28 +1493,36 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierInParentScopes(const Identif * Example: SELECT a AS b, b AS a; * Try resolve identifier in current scope: * 3. Try resolve identifier from expression arguments. - * If prefer_column_name_to_alias = true. * + * If prefer_column_name_to_alias = true. * 4. Try to resolve identifier from tables. * 5. Try to resolve identifier from aliases. * Otherwise. * 4. Try to resolve identifier from aliases. * 5. Try to resolve identifier from tables. * - * 6. If identifier is not resolved in current scope, try to resolve it in parent scopes. - * 7. If identifier was not resolved remove it from identifier_lookup_to_resolve_status table. + * 6. If it is table identifier lookup try to lookup identifier in current scope CTEs. + * + * 7. If identifier is not resolved in current scope, try to resolve it in parent scopes. + * 8. If identifier is not resolved from parent scopes and it is table identifier lookup try to lookup identifier + * in database catalog. + * + * Same is not done for functions because function resolution is more complex, and in case of aggregate functions requires not only name + * but also argument types, it is responsiblity of resolve function method to handle resolution of function name. + * + * 9. If identifier was not resolved remove it from identifier_lookup_to_resolve_status table. * * It is okay for identifier to be not resolved, in case we want first try to lookup identifier in one context, * then if there is no identifier in this context, try to lookup in another context. * Example: Try to lookup identifier as function, if it is not found lookup as expression. * Example: Try to lookup identifier as expression, if it is not found lookup as table. */ -QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifier(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, bool allow_to_check_parent_scopes) +IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, IdentifierResolveSettings identifier_resolve_settings) { - auto it = scope.identifier_lookup_to_node.find(identifier_lookup); - if (it != scope.identifier_lookup_to_node.end()) + auto it = scope.identifier_lookup_to_result.find(identifier_lookup); + if (it != scope.identifier_lookup_to_result.end()) { - if (!it->second) + if (!it->second.resolved_identifier) throw Exception(ErrorCodes::CYCLIC_ALIASES, "Cyclic aliases for identifier {}. In scope {}", identifier_lookup.identifier.getFullName(), @@ -1436,50 +1532,89 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifier(const IdentifierLookup & id return it->second; } - auto [insert_it, _] = scope.identifier_lookup_to_node.insert({identifier_lookup, QueryTreeNodePtr()}); + auto [insert_it, _] = scope.identifier_lookup_to_result.insert({identifier_lookup, IdentifierResolveResult()}); it = insert_it; /// Resolve identifier from current scope - auto resolved_identifier = tryResolveIdentifierFromExpressionArguments(identifier_lookup, scope); + IdentifierResolveResult resolve_result; + resolve_result.resolved_identifier = tryResolveIdentifierFromExpressionArguments(identifier_lookup, scope); + if (resolve_result.resolved_identifier) + resolve_result.resolve_place = IdentifierResolvePlace::EXPRESSION_ARGUMENTS; - if (!resolved_identifier) + if (!resolve_result.resolved_identifier) { bool prefer_column_name_to_alias = context->getSettingsRef().prefer_column_name_to_alias; if (unlikely(prefer_column_name_to_alias)) { - resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, scope); - if (!resolved_identifier) - resolved_identifier = tryResolveIdentifierFromAliases(identifier_lookup, scope, allow_to_check_parent_scopes); + resolve_result.resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, scope); + + if (resolve_result.resolved_identifier) + resolve_result.resolve_place = IdentifierResolvePlace::TABLES; + + if (!resolve_result.resolved_identifier) + { + resolve_result.resolved_identifier = tryResolveIdentifierFromAliases(identifier_lookup, scope, identifier_resolve_settings); + + if (resolve_result.resolved_identifier) + resolve_result.resolve_place = IdentifierResolvePlace::ALIASES; + } } else { - resolved_identifier = tryResolveIdentifierFromAliases(identifier_lookup, scope, allow_to_check_parent_scopes); + resolve_result.resolved_identifier = tryResolveIdentifierFromAliases(identifier_lookup, scope, identifier_resolve_settings); - if (!resolved_identifier) - resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, scope); + if (resolve_result.resolved_identifier) + { + resolve_result.resolve_place = IdentifierResolvePlace::ALIASES; + } + else + { + resolve_result.resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, scope); + + if (resolve_result.resolved_identifier) + resolve_result.resolve_place = IdentifierResolvePlace::TABLES; + } } } - if (!resolved_identifier && identifier_lookup.isTableLookup()) + if (!resolve_result.resolved_identifier && identifier_lookup.isTableLookup()) { auto cte_query_node_it = scope.cte_name_to_query_node.find(identifier_lookup.identifier.getFullName()); if (cte_query_node_it != scope.cte_name_to_query_node.end()) - resolved_identifier = cte_query_node_it->second; + { + resolve_result.resolved_identifier = cte_query_node_it->second; + resolve_result.resolve_place = IdentifierResolvePlace::CTE; + } } /// Try to resolve identifier from parent scopes - if (!resolved_identifier && allow_to_check_parent_scopes) - resolved_identifier = tryResolveIdentifierInParentScopes(identifier_lookup, scope); + if (!resolve_result.resolved_identifier && identifier_resolve_settings.allow_to_check_parent_scopes) + { + resolve_result = tryResolveIdentifierInParentScopes(identifier_lookup, scope); - it->second = resolved_identifier; + if (resolve_result.resolved_identifier) + resolve_result.resolved_from_parent_scopes = true; + } - if (!resolved_identifier || scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup)) - scope.identifier_lookup_to_node.erase(it); + /// Try to resolve table identifier from database catalog - return resolved_identifier; + if (!resolve_result.resolved_identifier && identifier_resolve_settings.allow_to_check_database_catalog && identifier_lookup.isTableLookup()) + { + resolve_result.resolved_identifier = tryResolveTableIdentifierFromDatabaseCatalog(identifier_lookup.identifier); + + if (resolve_result.resolved_identifier) + resolve_result.resolve_place = IdentifierResolvePlace::DATABASE_CATALOG; + } + + it->second = resolve_result; + + if (!resolve_result.resolved_identifier || scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup)) + scope.identifier_lookup_to_result.erase(it); + + return resolve_result; } /// Resolve query tree nodes functions implementation @@ -1505,7 +1640,10 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, if (matcher_node_typed.isQualified()) { - auto expression_query_tree_node = tryResolveIdentifier({matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::EXPRESSION}, scope); + auto expression_identifier_lookup = IdentifierLookup{matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::EXPRESSION}; + auto expression_identifier_resolve_result = tryResolveIdentifier(expression_identifier_lookup, scope); + auto expression_query_tree_node = expression_identifier_resolve_result.resolved_identifier; + if (expression_query_tree_node) { auto result_type = expression_query_tree_node->getResultType(); @@ -1540,9 +1678,16 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, } else { - auto table_expression_node = tryResolveIdentifier({matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::TABLE}, scope, false /*allow_to_check_parent_scopes*/); + IdentifierResolveSettings identifier_resolve_settings; + identifier_resolve_settings.allow_to_check_parent_scopes = false; + identifier_resolve_settings.allow_to_check_cte = false; + identifier_resolve_settings.allow_to_check_database_catalog = false; - if (!table_expression_node || (table_expression_node->as() && table_expression_node->as()->isCTE())) + auto table_identifier_lookup = IdentifierLookup{matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::TABLE}; + auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope, identifier_resolve_settings); + auto table_expression_node = table_identifier_resolve_result.resolved_identifier; + + if (!table_expression_node || table_identifier_resolve_result.resolve_place == IdentifierResolvePlace::DATABASE_CATALOG) { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Qualified matcher {} does not find table. In scope {}", @@ -1721,7 +1866,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, add_strict_transformer_column_name(replace_transformer, node_name); node = replace_expression; - resolveExpressionNode(node, scope, false /*allow_lambda_expression*/); + resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } } @@ -1879,7 +2024,7 @@ void QueryAnalyzer::resolveLambda(QueryTreeNodePtr & lambda_node, const QueryTre /** Lambda body expression is resolved as standard query expression node. * After that lambda is resolved, because its expression node is resolved. */ - resolveExpressionNode(lambda.getExpression(), scope, false /*allow_lambda_expression*/); + resolveExpressionNode(lambda.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); /** TODO: Lambda body can be resolved in expression list. And for standalone lambdas it will work. * TODO: It can potentially be resolved into table or another lambda. @@ -1919,9 +2064,6 @@ void QueryAnalyzer::resolveLambda(QueryTreeNodePtr & lambda_node, const QueryTre * they must be resolved. * 9. If function is suitable for constant folding, try to replace function node with constant result. * - * TODO: Special `in` function. - * TODO: Special `grouping` function. - * TODO: Window functions. */ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveScope & scope) { @@ -1929,9 +2071,11 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc if (function_node.isResolved()) return; + const auto & function_name = function_node.getFunctionName(); + /// Resolve function parameters - resolveExpressionNodeList(function_node.getParametersNode(), scope, false /*allow_lambda_expression*/); + resolveExpressionNodeList(function_node.getParametersNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); /// Convert function parameters into constant parameters array @@ -1953,9 +2097,11 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc parameters.push_back(constant_parameter); } + bool is_special_function_in = isNameOfInFunction(function_name); + /// Resolve function arguments - resolveExpressionNodeList(function_node.getArgumentsNode(), scope, true /*allow_lambda_expression*/); + resolveExpressionNodeList(function_node.getArgumentsNode(), scope, true /*allow_lambda_expression*/, is_special_function_in /*allow_table_expression*/); /// Initialize function argument columns @@ -1977,12 +2123,18 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc /** If function argument is lambda, save lambda argument index and initialize argument type as DataTypeFunction * where function argument types are initialized with empty array of lambda arguments size. */ - if (auto * lambda_query_tree_node = function_argument->as()) + if (const auto * lambda_node = function_argument->as()) { - size_t lambda_arguments_size = lambda_query_tree_node->getArguments().getNodes().size(); + size_t lambda_arguments_size = lambda_node->getArguments().getNodes().size(); argument_column.type = std::make_shared(DataTypes(lambda_arguments_size, nullptr), nullptr); function_lambda_arguments_indexes.push_back(function_argument_index); } + else if (is_special_function_in && (function_argument->getNodeType() == QueryTreeNodeType::TABLE || + function_argument->getNodeType() == QueryTreeNodeType::TABLE_FUNCTION || + function_argument->getNodeType() == QueryTreeNodeType::QUERY)) + { + argument_column.type = std::make_shared(); + } else { argument_column.type = function_argument->getResultType(); @@ -2006,7 +2158,8 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc /** Lookup function node name as lambda identifier. * If no lambda node exists with function node name identifier, try to resolve it as lambda from sql user defined functions. */ - auto lambda_expression_untyped = tryResolveIdentifier({Identifier{function_node.getFunctionName()}, IdentifierLookupContext::FUNCTION}, scope); + auto function_lookup_result = tryResolveIdentifier({Identifier{function_node.getFunctionName()}, IdentifierLookupContext::FUNCTION}, scope); + auto lambda_expression_untyped = function_lookup_result.resolved_identifier; if (!lambda_expression_untyped) lambda_expression_untyped = tryGetLambdaFromSQLUserDefinedFunction(function_node.getFunctionName()); @@ -2037,12 +2190,10 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc return; } - const auto & function_name = function_node.getFunctionName(); - - /// Special handling of `untuple` function - if (function_name == "untuple") { + /// Special handling of `untuple` function + if (function_arguments.size() != 1) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function 'untuple' must have 1 argument. In scope {}", @@ -2165,6 +2316,31 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc } } + /** Create SET column for special function IN to allow constant folding + * if left and right arguments are constants. + * + * Example: SELECT * FROM test_table LIMIT 1 IN 1; + */ + if (is_special_function_in && + function_arguments.at(0)->getNodeType() == QueryTreeNodeType::CONSTANT && + function_arguments.at(1)->getNodeType() == QueryTreeNodeType::CONSTANT) + { + const auto & first_argument = assert_cast(*function_arguments[0]); + const auto & second_argument = assert_cast(*function_arguments[1]); + + const auto & first_argument_type = first_argument.getResultType(); + const auto & second_argument_type = second_argument.getResultType(); + + const auto & second_argument_value = second_argument.getConstantValue(); + + auto set = makeSetForConstantValue(first_argument_type, second_argument_type, second_argument_value, context->getSettingsRef()); + + /// Create constant set column for constant folding + + auto column_set = ColumnSet::create(1, std::move(set)); + argument_columns[1].column = ColumnConst::create(std::move(column_set), 1); + } + DataTypePtr result_type; try @@ -2227,20 +2403,33 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc * SELECT value.value1 AS value FROM test_table; * * 2. Call specific resolve method depending on node type. - * TODO: Need a way to prevent scalar subquery evaluation, for special functions like `in`. - * 3. Special case identifier node. Try first resolve it as expression identifier. Then if allow_lambda_expression = true - * try to resolve it as function. TODO: Add tables. + * + * If allow_table_expression = true and node is query node, then it is not evaluated as scalar subquery. + * Althought if node is identifier that is resolved into query node that query is evaluated as scalar subquery. + * SELECT id, (SELECT 1) AS c FROM test_table WHERE a IN c; + * SELECT id, FROM test_table WHERE a IN (SELECT 1); + * + * 3. Special case identifier node. + * Try resolve it as expression identifier. + * Then if allow_lambda_expression = true try to resolve it as function. + * Then if allow_table_expression = true try to resolve it as table expression. + * * 4. If node has alias, update its value in scope alias map. Deregister alias from expression_aliases_in_resolve_process. */ -void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression) +void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) { String node_alias = node->getAlias(); /** Do not use alias table if node has alias same as some other node. * Example: WITH x -> x + 1 AS lambda SELECT 1 AS lambda; * During 1 AS lambda resolve if we use alias table we replace node with x -> x + 1 AS lambda. + * + * Do not use alias table if allow_table_expression = true and we resolve query node directly. + * Example: SELECT a FROM test_table WHERE id IN (SELECT 1) AS a; + * To support both (SELECT 1) AS expression in projection and (SELECT 1) as subquery in IN, do not use + * alias table because in alias table subquery could be evaluated as scalar. */ - bool use_alias_table = !scope.nodes_with_duplicated_aliases.contains(node); + bool use_alias_table = !scope.nodes_with_duplicated_aliases.contains(node) || (allow_table_expression && node->getNodeType() == QueryTreeNodeType::QUERY); if (!node_alias.empty() && use_alias_table) { @@ -2263,87 +2452,142 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes scope.expressions_in_resolve_process_stack.pushNode(node); - if (auto * identifier_node = node->as()) + auto node_type = node->getNodeType(); + + switch (node_type) { - auto unresolved_identifier = identifier_node->getIdentifier(); - node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::EXPRESSION}, scope); - - if (node && !node_alias.empty()) - scope.alias_name_to_lambda_node.erase(node_alias); - - if (!node && allow_lambda_expression) + case QueryTreeNodeType::IDENTIFIER: { - node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::FUNCTION}, scope); + auto & identifier_node = node->as(); + auto unresolved_identifier = identifier_node.getIdentifier(); + node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::EXPRESSION}, scope).resolved_identifier; if (node && !node_alias.empty()) - scope.alias_name_to_expression_node.erase(node_alias); - } + scope.alias_name_to_lambda_node.erase(node_alias); - if (!node) + if (!node && allow_lambda_expression) + { + node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::FUNCTION}, scope).resolved_identifier; + + if (node && !node_alias.empty()) + scope.alias_name_to_expression_node.erase(node_alias); + } + + if (!node && allow_table_expression) + { + node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::TABLE}, scope).resolved_identifier; + + if (node) + { + /// If table identifier is resolved as CTE clone it + bool resolved_as_cte = node->as() && node->as()->isCTE(); + + if (resolved_as_cte) + { + node = node->clone(); + node->as().setIsCTE(false); + } + } + } + + if (!node) + { + std::string message_clarification; + if (allow_lambda_expression) + message_clarification = std::string(" or ") + toStringLowercase(IdentifierLookupContext::FUNCTION); + + if (allow_table_expression) + message_clarification = std::string(" or ") + toStringLowercase(IdentifierLookupContext::TABLE); + + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Unknown {}{} identifier {} in scope {}", + toStringLowercase(IdentifierLookupContext::EXPRESSION), + message_clarification, + unresolved_identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + } + + node->setAlias(node_alias); + break; + } + case QueryTreeNodeType::MATCHER: { - std::string lambda_message_clarification; - if (allow_lambda_expression) - lambda_message_clarification = std::string(" or ") + toStringLowercase(IdentifierLookupContext::FUNCTION); - - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Unknown {}{} identifier {} in scope {}", - toStringLowercase(IdentifierLookupContext::EXPRESSION), - lambda_message_clarification, - unresolved_identifier.getFullName(), - scope.scope_node->formatASTForErrorMessage()); + node = resolveMatcher(node, scope); + break; } - - node->setAlias(node_alias); - } - else if (auto * function_node = node->as()) - { - resolveFunction(node, scope); - } - else if (auto * constant_function_node = node->as()) - { - /// Already resolved - } - else if (auto * column_function_node = node->as()) - { - if (column_function_node->hasAliasExpression()) - resolveExpressionNode(column_function_node->getAliasExpression(), scope, false /*allow_lambda_expression*/); - } - else if (auto * lambda_node = node->as()) - { - if (!allow_lambda_expression) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Lambda is not allowed {} in expression context. In scope {}", - lambda_node->formatASTForErrorMessage(), + case QueryTreeNodeType::TRANSFORMER: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Transformer {} appeared in expression context. In scope {}", + node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); + } + case QueryTreeNodeType::LIST: + { + /** Edge case if list expression has alias. + * Matchers cannot have aliases, but `untuple` function can. + * Example: SELECT a, untuple(CAST(('hello', 1) AS Tuple(name String, count UInt32))) AS a; + * During resolveFunction `untuple` function is replaced by list of 2 constants 'hello', 1. + */ + resolveExpressionNodeList(node, scope, allow_lambda_expression, allow_lambda_expression); + break; + } + case QueryTreeNodeType::CONSTANT: + { + /// Already resolved + break; + } + case QueryTreeNodeType::COLUMN: + { + auto & column_function_node = node->as(); + if (column_function_node.hasAliasExpression()) + resolveExpressionNode(column_function_node.getAliasExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + break; + } + case QueryTreeNodeType::FUNCTION: + { + resolveFunction(node, scope); + break; + } + case QueryTreeNodeType::LAMBDA: + { + if (!allow_lambda_expression) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Lambda is not allowed {} in expression. In scope {}", + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); - /// Must be resolved by caller - } - else if (auto * matcher_node = node->as()) - { - node = resolveMatcher(node, scope); - } - else if (auto * query_node = node->as()) - { - IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); - resolveQuery(node, subquery_scope); - evaluateScalarSubquery(node); - } - else if (auto * list_node = node->as()) - { - /** Edge case if list expression has alias. - * Matchers cannot have aliases, but `untuple` function can. - * Example: SELECT a, untuple(CAST(('hello', 1) AS Tuple(name String, count UInt32))) AS a; - * During resolveFunction `untuple` function is replaced by list of 2 constants 'hello', 1. - */ - resolveExpressionNodeList(node, scope, allow_lambda_expression); - } - else - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Node with type {} is not supported {}. In scope {}", - node->getNodeTypeName(), - node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); + /// Lambda must be resolved by caller + break; + } + case QueryTreeNodeType::TABLE: + { + if (!allow_table_expression) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Table is not allowed {} in expression. In scope {}", + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + break; + } + case QueryTreeNodeType::TABLE_FUNCTION: + { + if (!allow_table_expression) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Table function is not allowed {} in expression. In scope {}", + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + break; + } + case QueryTreeNodeType::QUERY: + { + IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); + resolveQuery(node, subquery_scope); + + if (!allow_table_expression) + evaluateScalarSubquery(node); + + break; + } } /** Update aliases after expression node was resolved. @@ -2377,7 +2621,7 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes * Example: CREATE TABLE test_table (id UInt64, value UInt64) ENGINE=TinyLog; SELECT plus(*) FROM test_table; * Example: SELECT *** FROM system.one; */ -void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression) +void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) { auto & initial_node_list = node_list->as(); size_t initial_node_list_size = initial_node_list.getNodes().size(); @@ -2391,7 +2635,7 @@ void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, Iden if (node->as() && node->as()->isCTE()) continue; - resolveExpressionNode(node, scope, allow_lambda_expression); + resolveExpressionNode(node, scope, allow_lambda_expression, allow_table_expression); if (auto * expression_list = node->as()) { @@ -2422,62 +2666,28 @@ void QueryAnalyzer::initializeQueryFrom(QueryTreeNodePtr & from_node, Identifier } else if (auto * from_table_identifier = from_node->as()) { - /// TODO: Context resolve storage + auto table_identifier_lookup = IdentifierLookup{from_table_identifier->getIdentifier(), IdentifierLookupContext::TABLE}; + auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope); + auto resolved_identifier = table_identifier_resolve_result.resolved_identifier; - const auto & storage_identifier = from_table_identifier->getIdentifier(); - size_t parts_size = storage_identifier.getPartsSize(); - if (parts_size < 1 || parts_size > 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table identifier should consist of 1 or 2 parts. Actual {}", storage_identifier.getFullName()); + if (!resolved_identifier) + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Unknown {} identifier {} in scope {}", + toStringLowercase(IdentifierLookupContext::TABLE), + from_table_identifier->getIdentifier().getFullName(), + scope.scope_node->formatASTForErrorMessage()); - std::string database_name; - std::string table_name; + bool resolved_as_cte = resolved_identifier->as() && resolved_identifier->as()->isCTE(); - if (storage_identifier.isCompound()) + if (resolved_as_cte) { - database_name = storage_identifier[0]; - table_name = storage_identifier[1]; - } - else - { - table_name = storage_identifier[0]; + resolved_identifier = resolved_identifier->clone(); + resolved_identifier->as().setIsCTE(false); } - bool resolved_as_cte = false; - - if (database_name.empty()) - { - auto table_identifier_lookup = IdentifierLookup{Identifier(table_name), IdentifierLookupContext::TABLE}; - auto resolved_identifier = tryResolveIdentifier(table_identifier_lookup, scope, true /*allow_to_check_parent_scopes*/); - - if (resolved_identifier && resolved_identifier->as() && resolved_identifier->as()->isCTE()) - { - auto from_node_alias = from_node->getAlias(); - from_node = resolved_identifier->clone(); - from_node->as().setIsCTE(false); - from_node->setAlias(from_node_alias); - - resolved_as_cte = true; - } - - database_name = context->getCurrentDatabase(); - } - - if (!resolved_as_cte) - { - auto & database_catalog = DatabaseCatalog::instance(); - auto database = database_catalog.tryGetDatabase(database_name); - if (!database) - throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exists", database_name); - - auto table = database->tryGetTable(table_name, context); - - if (!table) - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exists", table_name); - - auto from_node_alias = from_node->getAlias(); - from_node = std::make_shared(std::move(table), context); - from_node->setAlias(from_node_alias); - } + auto from_node_alias = from_node->getAlias(); + resolved_identifier->setAlias(from_node_alias); + from_node = resolved_identifier; } else { @@ -2551,7 +2761,7 @@ void QueryAnalyzer::resolveQueryFrom(QueryTreeNodePtr & from_node, IdentifierRes scope.scope_node->formatASTForErrorMessage()); } - resolveExpressionNode(argument_node, scope, false /*allow_lambda_expression*/); + resolveExpressionNode(argument_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } auto table_function_ast = from_table_function->toAST(); @@ -2648,20 +2858,20 @@ void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_node, IdentifierResolv * During FROM section resolve identifier a is resolved into CTE. * During qualified matcher a.* resolve identifier a must be resolved into table. */ - scope.identifier_lookup_to_node.clear(); + scope.identifier_lookup_to_result.clear(); /// Resolve query node sections. if (query_node_typed.getWithNode()) - resolveExpressionNodeList(query_node_typed.getWithNode(), scope, true /*allow_lambda_expression*/); + resolveExpressionNodeList(query_node_typed.getWithNode(), scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); - resolveExpressionNodeList(query_node_typed.getProjectionNode(), scope, false /*allow_lambda_expression*/); + resolveExpressionNodeList(query_node_typed.getProjectionNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); if (query_node_typed.getPrewhere()) - resolveExpressionNode(query_node_typed.getPrewhere(), scope, false /*allow_lambda_expression*/); + resolveExpressionNode(query_node_typed.getPrewhere(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); if (query_node_typed.getWhere()) - resolveExpressionNode(query_node_typed.getWhere(), scope, false /*allow_lambda_expression*/); + resolveExpressionNode(query_node_typed.getWhere(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); /** WITH section can be safely removed, because WITH section only can provide aliases to expressions * and CTE for other sections to use. @@ -2671,6 +2881,7 @@ void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_node, IdentifierResolv query_node_typed.getWithNode() = std::make_shared(); /** Resolve nodes with duplicate aliases. + * Table expressions cannot have duplicate aliases. * * Such nodes during scope aliases collection are placed into duplicated array. * After scope nodes are resolved, we can compare node with duplicate alias with @@ -2680,7 +2891,7 @@ void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_node, IdentifierResolv { auto node = node_with_duplicated_alias; auto node_alias = node->getAlias(); - resolveExpressionNode(node, scope, true /*allow_lambda_expression*/); + resolveExpressionNode(node, scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); bool has_node_in_alias_table = false; diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index 78da924e96f..bf56b739a90 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -31,7 +31,8 @@ namespace DB * Function `arrayJoin` is handled properly. * For function `grouping` arguments are resolved, but it is planner responsibility to initialize it with concrete grouping function * based on group by kind and group by keys positions. - * 9. Sets are build for `in` function. + * For function `in` and its variations arguments are resolved, but sets are not build. If left and right arguments are constants + * constant folding is performed. */ class QueryAnalysisPass final : public IQueryTreePass { diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index a9584752e6d..83a2d6d2698 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -22,6 +22,39 @@ QueryNode::QueryNode() children[projection_child_index] = std::make_shared(); } +String QueryNode::getName() const +{ + WriteBufferFromOwnString buffer; + + if (!getWith().getNodes().empty()) + { + buffer << getWith().getName(); + } + + buffer << " SELECT "; + buffer << getProjection().getName(); + + if (getFrom()) + { + buffer << " FROM "; + buffer << getFrom()->getName(); + } + + if (getPrewhere()) + { + buffer << " PREWHERE "; + buffer << getPrewhere()->getName(); + } + + if (getWhere()) + { + buffer << " WHERE "; + buffer << getWhere()->getName(); + } + + return buffer.str(); +} + void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "QUERY id: " << format_state.getNodeId(this); diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 6d08bf0642f..78941cb2cae 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -122,6 +122,8 @@ public: return QueryTreeNodeType::QUERY; } + String getName() const override; + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; bool isEqualImpl(const IQueryTreeNode & rhs) const override; @@ -147,7 +149,7 @@ private: static constexpr size_t having_child_index = 6; static constexpr size_t order_by_child_index = 7; static constexpr size_t limit_child_index = 8; - static constexpr size_t children_size = where_child_index + 1; + static constexpr size_t children_size = limit_child_index + 1; }; } diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp new file mode 100644 index 00000000000..ea456f3e5fe --- /dev/null +++ b/src/Analyzer/SetUtils.cpp @@ -0,0 +1,183 @@ +#include + +#include + +#include +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_ELEMENT_OF_SET; +} + +namespace +{ + +size_t getCompoundTypeDepth(const IDataType & type) +{ + size_t result = 0; + + const IDataType * current_type = &type; + + while (true) + { + WhichDataType which_type(*current_type); + + if (which_type.isArray()) + { + current_type = &(*assert_cast(*current_type).getNestedType()); + ++result; + } + else if (which_type.isTuple()) + { + const auto & tuple_elements = assert_cast(*current_type).getElements(); + if (!tuple_elements.empty()) + current_type = &(*assert_cast(*current_type).getElements().at(0)); + + ++result; + } + else + { + break; + } + } + + return result; +} + +template +Block createBlockFromCollection(const Collection & collection, const DataTypes & block_types, bool transform_null_in) +{ + size_t columns_size = block_types.size(); + MutableColumns columns(columns_size); + for (size_t i = 0; i < columns_size; ++i) + { + columns[i] = block_types[i]->createColumn(); + columns[i]->reserve(collection.size()); + } + + Row tuple_values; + + for (const auto & value : collection) + { + if (columns_size == 1) + { + auto field = convertFieldToType(value, *block_types[0]); + bool need_insert_null = transform_null_in && block_types[0]->isNullable(); + if (!field.isNull() || need_insert_null) + columns[0]->insert(std::move(field)); + + continue; + } + + if (value.getType() != Field::Types::Tuple) + throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET, + "Invalid type in set. Expected tuple, got {}", + value.getTypeName()); + + const auto & tuple = DB::get(value); + size_t tuple_size = tuple.size(); + + if (tuple_size != columns_size) + throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET, + "Incorrect size of tuple in set: {} instead of {}", + toString(tuple_size), + toString(columns_size)); + + if (tuple_values.empty()) + tuple_values.resize(tuple_size); + + size_t i = 0; + for (; i < tuple_size; ++i) + { + tuple_values[i] = convertFieldToType(tuple[i], *block_types[i]); + bool need_insert_null = transform_null_in && block_types[i]->isNullable(); + if (tuple_values[i].isNull() && !need_insert_null) + break; + } + + if (i == tuple_size) + for (i = 0; i < tuple_size; ++i) + columns[i]->insert(tuple_values[i]); + } + + Block res; + for (size_t i = 0; i < columns_size; ++i) + res.insert(ColumnWithTypeAndName{std::move(columns[i]), block_types[i], "argument_" + toString(i)}); + + return res; +} + +} + +SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const DataTypePtr & value_type, const Field & value, const Settings & settings) +{ + DataTypes set_element_types = {expression_type}; + const auto * lhs_tuple_type = typeid_cast(expression_type.get()); + + if (lhs_tuple_type && lhs_tuple_type->getElements().size() != 1) + set_element_types = lhs_tuple_type->getElements(); + + for (auto & set_element_type : set_element_types) + { + if (const auto * set_element_low_cardinality_type = typeid_cast(set_element_type.get())) + set_element_type = set_element_low_cardinality_type->getDictionaryType(); + } + + size_t lhs_type_depth = getCompoundTypeDepth(*expression_type); + size_t rhs_type_depth = getCompoundTypeDepth(*value_type); + + SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; + bool tranform_null_in = settings.transform_null_in; + + Block result_block; + + if (lhs_type_depth == rhs_type_depth) + { + /// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc. + + Array array{value}; + result_block = createBlockFromCollection(array, set_element_types, tranform_null_in); + } + else if (lhs_type_depth + 1 == rhs_type_depth) + { + /// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)) + + WhichDataType rhs_which_type(value_type); + + if (rhs_which_type.isArray()) + result_block = createBlockFromCollection(value.get(), set_element_types, tranform_null_in); + else if (rhs_which_type.isTuple()) + { + result_block = createBlockFromCollection(value.get(), set_element_types, tranform_null_in); + } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Unsupported type at the right-side of IN. Expected Array or Tuple. Actual {}", + value_type->getName()); + } + else + { + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Unsupported types for IN. First argument type {}. Second argument type {}", + expression_type->getName(), + value_type->getName()); + } + + auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); + + set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); + set->insertFromBlock(result_block.getColumnsWithTypeAndName()); + set->finishInsert(); + + return set; +} + +} diff --git a/src/Analyzer/SetUtils.h b/src/Analyzer/SetUtils.h new file mode 100644 index 00000000000..e39e388a1fb --- /dev/null +++ b/src/Analyzer/SetUtils.h @@ -0,0 +1,30 @@ +#pragma once + +#include + +#include + +#include + +namespace DB +{ + +class Set; +using SetPtr = std::shared_ptr; + +/** Make set for constant part of IN subquery. + * Throws exception if parameters are not valid for IN function. + * + * Example: SELECT id FROM test_table WHERE id IN (1, 2, 3, 4); + * Example: SELECT id FROM test_table WHERE id IN ((1, 2), (3, 4)); + * + * @param expression_type - type of first argument of function IN. + * @param value_type - type of second argument of function IN. + * @param value - constant value of second argument of function IN. + * @param settings - query settings. + * + * @return SetPtr for constant value. + */ +SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const DataTypePtr & value_type, const Field & value, const Settings & settings); + +} diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp new file mode 100644 index 00000000000..1aade42351f --- /dev/null +++ b/src/Analyzer/Utils.cpp @@ -0,0 +1,39 @@ +#include + +namespace DB +{ + +bool isNodePartOfTree(const IQueryTreeNode * node, const IQueryTreeNode * root) +{ + std::vector nodes_to_process; + nodes_to_process.push_back(root); + + while (!nodes_to_process.empty()) + { + const auto * subtree_node = nodes_to_process.back(); + nodes_to_process.pop_back(); + + if (subtree_node == node) + return true; + + for (const auto & child : subtree_node->getChildren()) + { + if (child) + nodes_to_process.push_back(child.get()); + } + } + + return false; +} + +bool isNameOfInFunction(const std::string & function_name) +{ + bool is_special_function_in = function_name == "in" || function_name == "globalIn" || function_name == "notIn" || function_name == "globalNotIn" || + function_name == "nullIn" || function_name == "globalNullIn" || function_name == "notNullIn" || function_name == "globalNotNullIn" || + function_name == "inIgnoreSet" || function_name == "globalInIgnoreSet" || function_name == "notInIgnoreSet" || function_name == "globalNotInIgnoreSet" || + function_name == "nullInIgnoreSet" || function_name == "globalNullInIgnoreSet" || function_name == "notNullInIgnoreSet" || function_name == "globalNotNullInIgnoreSet"; + + return is_special_function_in; +} + +} diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h new file mode 100644 index 00000000000..d93d81d654e --- /dev/null +++ b/src/Analyzer/Utils.h @@ -0,0 +1,14 @@ +#pragma once + +#include + +namespace DB +{ + +/// Returns true if node part of root tree, false otherwise +bool isNodePartOfTree(const IQueryTreeNode * node, const IQueryTreeNode * root); + +/// Returns true if function name is name of IN function or its variations, false otherwise +bool isNameOfInFunction(const std::string & function_name); + +} diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 5fe7b9c1f20..b407a77c50c 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -13,6 +13,7 @@ #include #include +#include #include #include #include @@ -390,6 +391,10 @@ private: throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function GROUPING is not supported"); } + else if (isNameOfInFunction(function_node.getFunctionName())) + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function IN is not supported"); + } if (function_node.isAggregateFunction()) { diff --git a/tests/queries/0_stateless/02370_analyzer_in_function.reference b/tests/queries/0_stateless/02370_analyzer_in_function.reference new file mode 100644 index 00000000000..fda174c0b7c --- /dev/null +++ b/tests/queries/0_stateless/02370_analyzer_in_function.reference @@ -0,0 +1,14 @@ +1 +1 +0 +0 +1 +1 +0 +1 +-- +1 +0 +1 +1 +0 diff --git a/tests/queries/0_stateless/02370_analyzer_in_function.sql b/tests/queries/0_stateless/02370_analyzer_in_function.sql new file mode 100644 index 00000000000..01e5cf498cd --- /dev/null +++ b/tests/queries/0_stateless/02370_analyzer_in_function.sql @@ -0,0 +1,23 @@ +SET use_analyzer = 1; + +SELECT 1 IN 1; +SELECT 1 IN (1); +SELECT 1 IN 0; +SELECT 1 IN (0); +SELECT 1 IN (1, 2); +SELECT (1, 1) IN ((1, 1), (1, 2)); +SELECT (1, 1) IN ((1, 2), (1, 2)); +SELECT 1 IN (((1), (2))); + +SELECT '--'; + +SELECT 1 IN [1]; +SELECT 1 IN [0]; +SELECT 1 IN [1, 2]; +SELECT (1, 1) IN [(1, 1), (1, 2)]; +SELECT (1, 1) IN [(1, 2), (1, 2)]; + +SELECT (1, 2) IN 1; -- { serverError 43 } +SELECT (1, 2) IN [1]; -- { serverError 124 } +SELECT (1, 2) IN (((1, 2), (1, 2)), ((1, 2), (1, 2))); -- { serverError 43 } +SELECT (1, 2) IN [((1, 2), (1, 2)), ((1, 2), (1, 2))]; -- { serverError 43 } From 1717d16fa4fd55eb12d10f4735a04ff987a34bb3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 22 Jul 2022 18:31:54 +0200 Subject: [PATCH 065/188] Improved lambda matcher support --- src/Analyzer/QueryAnalysisPass.cpp | 54 +++++++++++++------ .../02343_analyzer_lambdas.reference | 4 ++ .../0_stateless/02343_analyzer_lambdas.sql | 9 ++++ 3 files changed, 51 insertions(+), 16 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index c5dc498eba5..d20e8598e03 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -591,7 +591,7 @@ private: QueryTreeNodePtr resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); - void resolveLambda(QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope); + void resolveLambda(const QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope); void resolveFunction(QueryTreeNodePtr & function_node, IdentifierResolveScope & scope); @@ -603,7 +603,7 @@ private: void resolveQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope); - void resolveQuery(QueryTreeNodePtr & query_node, IdentifierResolveScope & scope); + void resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope); /// Query analyzer context ContextPtr context; @@ -1824,6 +1824,22 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope lambda_scope(expression_node, &scope /*parent_scope*/); resolveLambda(lambda_expression_to_resolve, {node}, lambda_scope); auto & lambda_expression_to_resolve_typed = lambda_expression_to_resolve->as(); + + if (auto * lambda_list_node_result = lambda_expression_to_resolve_typed.getExpression()->as()) + { + auto & lambda_list_node_result_nodes = lambda_list_node_result->getNodes(); + size_t lambda_list_node_result_nodes_size = lambda_list_node_result->getNodes().size(); + + if (lambda_list_node_result_nodes_size != 1) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Lambda in APPLY transformer {} resolved as list node with size {}. Expected 1. In scope {}", + apply_transformer->formatASTForErrorMessage(), + lambda_list_node_result_nodes_size, + scope.scope_node->formatASTForErrorMessage()); + + lambda_expression_to_resolve_typed.getExpression() = lambda_list_node_result_nodes[0]; + } + node = lambda_expression_to_resolve_typed.getExpression(); } else if (apply_transformer->getApplyTransformerType() == ApplyColumnTransformerType::FUNCTION) @@ -1948,6 +1964,8 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, * This function modified lambda_node during resolve. It is caller responsibility to clone lambda before resolve * if it is needed for later use. * + * Lambda expression can be resolved into list node. It is caller responsibility to handle it properly. + * * lambda_node - node that must have LambdaNode type. * arguments - lambda arguments. * scope - lambda scope. It is client responsibility to create it. @@ -1960,7 +1978,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, * 5. Resolve lambda body expression. * 6. Deregister lambda from lambdas in resolve process. */ -void QueryAnalyzer::resolveLambda(QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope) +void QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope) { auto & lambda = lambda_node->as(); auto & lambda_arguments_nodes = lambda.getArguments().getNodes(); @@ -1988,7 +2006,7 @@ void QueryAnalyzer::resolveLambda(QueryTreeNodePtr & lambda_node, const QueryTre /// Initialize aliases in lambda scope ScopeAliasVisitorMatcher::Data data{scope}; ScopeAliasVisitorMatcher::Visitor visitor(data); - visitor.visit(lambda_node); + visitor.visit(lambda.getExpression()); /** Replace lambda arguments with new arguments. * Additionally validate that there are no aliases with same name as lambda arguments. @@ -2026,16 +2044,6 @@ void QueryAnalyzer::resolveLambda(QueryTreeNodePtr & lambda_node, const QueryTre */ resolveExpressionNode(lambda.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - /** TODO: Lambda body can be resolved in expression list. And for standalone lambdas it will work. - * TODO: It can potentially be resolved into table or another lambda. - * Example: WITH (x -> untuple(x)) AS lambda SELECT untuple(compound_expression). - */ - // if (lambda.getExpression()->getNodeType() == QueryTreeNodeType::LIST) - // throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - // "Lambda {} expression body cannot contain list of expressions. In scope {}", - // lambda_node->formatASTForErrorMessage(), - // scope.scope_node->formatASTForErrorMessage()); - lambdas_in_resolve_process.erase(lambda_node.get()); } @@ -2290,7 +2298,7 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc size_t function_data_type_arguments_size = function_data_type_argument_types.size(); if (function_data_type_arguments_size != lambda_arguments_size) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Function {} function data type for lambda argument wiht index {} arguments size mismatch. Actual {}. Expected {}. In scope {}", + "Function {} function data type for lambda argument with index {} arguments size mismatch. Actual {}. Expected {}. In scope {}", function_name, function_data_type_arguments_size, lambda_arguments_size, @@ -2310,6 +2318,20 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc IdentifierResolveScope lambda_scope(lambda_to_resolve, &scope /*parent_scope*/); resolveLambda(lambda_to_resolve, lambda_arguments, lambda_scope); + if (auto * lambda_list_node_result = lambda_to_resolve_typed.getExpression()->as()) + { + auto & lambda_list_node_result_nodes = lambda_list_node_result->getNodes(); + size_t lambda_list_node_result_nodes_size = lambda_list_node_result->getNodes().size(); + + if (lambda_list_node_result_nodes_size != 1) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Lambda as function argument resolved as list node with size {}. Expected 1. In scope {}", + lambda_list_node_result_nodes_size, + lambda_to_resolve->formatASTForErrorMessage()); + + lambda_to_resolve_typed.getExpression() = lambda_list_node_result_nodes[0]; + } + argument_types[function_lambda_argument_index] = std::make_shared(function_data_type_argument_types, lambda_to_resolve->getResultType()); argument_columns[function_lambda_argument_index].type = argument_types[function_lambda_argument_index]; function_arguments[function_lambda_argument_index] = std::move(lambda_to_resolve); @@ -2795,7 +2817,7 @@ void QueryAnalyzer::resolveQueryFrom(QueryTreeNodePtr & from_node, IdentifierRes * 5. Remove WITH section from query. * 6. Validate nodes with duplicate aliases. */ -void QueryAnalyzer::resolveQuery(QueryTreeNodePtr & query_node, IdentifierResolveScope & scope) +void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope) { auto & query_node_typed = query_node->as(); diff --git a/tests/queries/0_stateless/02343_analyzer_lambdas.reference b/tests/queries/0_stateless/02343_analyzer_lambdas.reference index 4bfd09df731..8d29481c255 100644 --- a/tests/queries/0_stateless/02343_analyzer_lambdas.reference +++ b/tests/queries/0_stateless/02343_analyzer_lambdas.reference @@ -18,6 +18,10 @@ value_0_level_0_value_1_level_0 Lambda matcher 0 0 Value +[1,1,1] +[2,2,2] +0 1 1 +0 2 2 Lambda untuple (1,'Value') 1 Value Lambda carrying diff --git a/tests/queries/0_stateless/02343_analyzer_lambdas.sql b/tests/queries/0_stateless/02343_analyzer_lambdas.sql index 10623bdaa34..97e86b0ee39 100644 --- a/tests/queries/0_stateless/02343_analyzer_lambdas.sql +++ b/tests/queries/0_stateless/02343_analyzer_lambdas.sql @@ -48,6 +48,14 @@ SELECT 'Lambda matcher'; WITH x -> * AS lambda SELECT lambda(1); WITH x -> * AS lambda SELECT lambda(1) FROM test_table; +WITH cast(tuple(1), 'Tuple (value UInt64)') AS compound_value SELECT arrayMap(x -> compound_value.*, [1,2,3]); +WITH cast(tuple(1, 1), 'Tuple (value_1 UInt64, value_2 UInt64)') AS compound_value SELECT arrayMap(x -> compound_value.*, [1,2,3]); -- { serverError 1 } +WITH cast(tuple(1, 1), 'Tuple (value_1 UInt64, value_2 UInt64)') AS compound_value SELECT arrayMap(x -> plus(compound_value.*), [1,2,3]); + +WITH cast(tuple(1), 'Tuple (value UInt64)') AS compound_value SELECT id, test_table.* APPLY x -> compound_value.* FROM test_table; +WITH cast(tuple(1, 1), 'Tuple (value_1 UInt64, value_2 UInt64)') AS compound_value SELECT id, test_table.* APPLY x -> compound_value.* FROM test_table; -- { serverError 1 } +WITH cast(tuple(1, 1), 'Tuple (value_1 UInt64, value_2 UInt64)') AS compound_value SELECT id, test_table.* APPLY x -> plus(compound_value.*) FROM test_table; + SELECT 'Lambda untuple'; WITH x -> untuple(x) AS lambda SELECT cast((1, 'Value'), 'Tuple (id UInt64, value String)') AS value, lambda(value); @@ -57,4 +65,5 @@ SELECT 'Lambda carrying'; WITH functor, x -> functor(x) AS lambda, x -> x + 1 AS functor_1, x -> toString(x) AS functor_2 SELECT lambda(functor_1, 1), lambda(functor_2, 1); WITH functor, x -> functor(x) AS lambda, x -> x + 1 AS functor_1, x -> toString(x) AS functor_2 SELECT lambda(functor_1, id), lambda(functor_2, id) FROM test_table; +DROP TABLE test_table_tuple; DROP TABLE test_table; From 40cb904a3d5d85e35d718f433d521a924fd4ad95 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 15 Aug 2022 18:34:10 +0200 Subject: [PATCH 066/188] Added support for JOIN, ARRAY JOIN --- src/Analyzer/ArrayJoinNode.cpp | 76 + src/Analyzer/ArrayJoinNode.h | 105 ++ src/Analyzer/ColumnNode.cpp | 48 +- src/Analyzer/ColumnNode.h | 45 +- src/Analyzer/IQueryTreeNode.cpp | 2 + src/Analyzer/IQueryTreeNode.h | 2 + src/Analyzer/JoinNode.cpp | 159 ++ src/Analyzer/JoinNode.h | 148 ++ src/Analyzer/ListNode.cpp | 5 + src/Analyzer/ListNode.h | 6 + src/Analyzer/MatcherNode.cpp | 2 + src/Analyzer/QueryAnalysisPass.cpp | 1225 +++++++----- src/Analyzer/QueryNode.cpp | 56 +- src/Analyzer/QueryNode.h | 8 + src/Analyzer/QueryTreeBuilder.cpp | 93 +- src/Analyzer/TableNode.cpp | 8 +- src/Analyzer/TableNode.h | 16 +- src/Analyzer/Utils.cpp | 94 + src/Analyzer/Utils.h | 8 + src/Analyzer/examples/query_analyzer.cpp | 2 - src/Interpreters/ExpressionActions.cpp | 4 +- src/Interpreters/InterpreterExplainQuery.cpp | 28 +- .../InterpreterSelectQueryAnalyzer.cpp | 1636 +++++++++++++++-- .../InterpreterSelectQueryAnalyzer.h | 7 +- src/Interpreters/TableJoin.cpp | 153 +- src/Interpreters/TableJoin.h | 44 +- src/Interpreters/TreeRewriter.cpp | 17 +- .../02337_analyzer_columns_basic.sql | 1 + ...369_analyzer_array_join_function.reference | 38 - .../02369_analyzer_array_join_function.sql | 9 +- .../02371_analyzer_join_cross.reference | 133 ++ .../0_stateless/02371_analyzer_join_cross.sql | 78 + .../02372_analyzer_join_inner.reference | 51 + .../0_stateless/02372_analyzer_join_inner.sql | 156 ++ 34 files changed, 3638 insertions(+), 825 deletions(-) create mode 100644 src/Analyzer/ArrayJoinNode.cpp create mode 100644 src/Analyzer/ArrayJoinNode.h create mode 100644 src/Analyzer/JoinNode.cpp create mode 100644 src/Analyzer/JoinNode.h create mode 100644 tests/queries/0_stateless/02371_analyzer_join_cross.reference create mode 100644 tests/queries/0_stateless/02371_analyzer_join_cross.sql create mode 100644 tests/queries/0_stateless/02372_analyzer_join_inner.reference create mode 100644 tests/queries/0_stateless/02372_analyzer_join_inner.sql diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp new file mode 100644 index 00000000000..c85f52b1657 --- /dev/null +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -0,0 +1,76 @@ +#include + +#include +#include +#include + +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + +ArrayJoinNode::ArrayJoinNode(QueryTreeNodePtr table_expression_, QueryTreeNodePtr join_expressions_, bool is_left_) + : is_left(is_left_) +{ + children.resize(children_size); + children[table_expression_child_index] = std::move(table_expression_); + children[join_expressions_child_index] = std::move(join_expressions_); +} + +void ArrayJoinNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const +{ + buffer << std::string(indent, ' ') << "ARRAY_JOIN id: " << format_state.getNodeId(this); + buffer << ", is_left: " << is_left; + + buffer << '\n' << std::string(indent + 2, ' ') << "TABLE EXPRESSION\n"; + getTableExpression()->dumpTreeImpl(buffer, format_state, indent + 4); + + buffer << '\n' << std::string(indent + 2, ' ') << "JOIN EXPRESSSION\n"; + getJoinExpressionsNode()->dumpTreeImpl(buffer, format_state, indent + 4); +} + +bool ArrayJoinNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + return is_left == rhs_typed.is_left; +} + +void ArrayJoinNode::updateTreeHashImpl(HashState & state) const +{ + state.update(is_left); +} + +ASTPtr ArrayJoinNode::toASTImpl() const +{ + auto array_join_ast = std::make_shared(); + array_join_ast->kind = is_left ? ASTArrayJoin::Kind::Left : ASTArrayJoin::Kind::Inner; + const auto & join_expression_list_node = getJoinExpressionsNode(); + array_join_ast->children.push_back(join_expression_list_node->toAST()); + array_join_ast->expression_list = array_join_ast->children.back(); + + ASTPtr tables_in_select_query_ast = std::make_shared(); + addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[table_expression_child_index]); + + auto array_join_query_element_ast = std::make_shared(); + array_join_query_element_ast->children.push_back(std::move(array_join_ast)); + array_join_query_element_ast->array_join = array_join_query_element_ast->children.back(); + + tables_in_select_query_ast->children.push_back(std::move(array_join_query_element_ast)); + + return tables_in_select_query_ast; +} + +QueryTreeNodePtr ArrayJoinNode::cloneImpl() const +{ + ArrayJoinNodePtr result_array_join_node(new ArrayJoinNode(is_left)); + return result_array_join_node; +} + +} diff --git a/src/Analyzer/ArrayJoinNode.h b/src/Analyzer/ArrayJoinNode.h new file mode 100644 index 00000000000..248e8443143 --- /dev/null +++ b/src/Analyzer/ArrayJoinNode.h @@ -0,0 +1,105 @@ +#pragma once + +#include +#include +#include + +#include +#include + +#include +#include + +namespace DB +{ + +/** Array join node represents array join in query tree. + * + * In query tree join expression is represented by list query tree node. + * + * Example: SELECT id FROM test_table ARRAY JOIN [1, 2, 3] as a. + * Example: SELECT id FROM test_table ARRAY JOIN [1, 2, 3] as a, [4, 5, 6] as b. + */ +class ArrayJoinNode; +using ArrayJoinNodePtr = std::shared_ptr; + +class ArrayJoinNode final : public IQueryTreeNode +{ +public: + /** Construct array join node with table expression. + * Example: SELECT id FROM test_table ARRAY JOIN [1, 2, 3] as a. + * test_table - table expression. + * join_expression_list - list of array join expressions. + */ + ArrayJoinNode(QueryTreeNodePtr table_expression_, QueryTreeNodePtr join_expressions_, bool is_left_); + + /// Get table expression + const QueryTreeNodePtr & getTableExpression() const + { + return children[table_expression_child_index]; + } + + /// Get table expression + QueryTreeNodePtr & getTableExpression() + { + return children[table_expression_child_index]; + } + + /// Get join expressions + const ListNode & getJoinExpressions() const + { + return children[join_expressions_child_index]->as(); + } + + /// Get join expressions + ListNode & getJoinExpressions() + { + return children[join_expressions_child_index]->as(); + } + + /// Get join expressions node + const QueryTreeNodePtr & getJoinExpressionsNode() const + { + return children[join_expressions_child_index]; + } + + QueryTreeNodePtr & getJoinExpressionsNode() + { + return children[join_expressions_child_index]; + } + + bool isLeft() const + { + return is_left; + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::ARRAY_JOIN; + } + + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + + void updateTreeHashImpl(HashState & state) const override; + +protected: + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + explicit ArrayJoinNode(bool is_left_) + : is_left(is_left_) + {} + + bool is_left = false; + + static constexpr size_t table_expression_child_index = 0; + static constexpr size_t join_expressions_child_index = 1; + static constexpr size_t children_size = join_expressions_child_index + 1; +}; + +} + diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index 28023f7328a..5fdce4c574c 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -20,15 +20,31 @@ ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_sour : column(std::move(column_)) , column_source(std::move(column_source_)) { - children.resize(1); + children.resize(children_size); } -ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodePtr alias_expression_node_, QueryTreeNodeWeakPtr column_source_) +ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_) : column(std::move(column_)) , column_source(std::move(column_source_)) { - children.resize(1); - children[0] = std::move(alias_expression_node_); + children.resize(children_size); + children[expression_child_index] = std::move(expression_node_); +} + +ColumnNode::ColumnNode(NameAndTypePair column_, String column_name_qualification_, QueryTreeNodeWeakPtr column_source_) + : column(std::move(column_)) + , column_name_qualification(std::move(column_name_qualification_)) + , column_source(std::move(column_source_)) +{ +} + +ColumnNode::ColumnNode(NameAndTypePair column_, String column_name_qualification_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_) + : column(std::move(column_)) + , column_name_qualification(std::move(column_name_qualification_)) + , column_source(std::move(column_source_)) +{ + children.resize(children_size); + children[expression_child_index] = std::move(expression_node_); } QueryTreeNodePtr ColumnNode::getColumnSource() const @@ -52,23 +68,26 @@ void ColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t buffer << ", column_name: " << column.name << ", result_type: " << column.type->getName(); + if (!column_name_qualification.empty()) + buffer << ", column_name_qualification: " << column_name_qualification; + auto column_source_ptr = column_source.lock(); if (column_source_ptr) buffer << ", source_id: " << state.getNodeId(column_source_ptr.get()); - const auto & alias_expression = getAliasExpression(); + const auto & expression = getExpression(); - if (alias_expression) + if (expression) { - buffer << '\n' << std::string(indent + 2, ' ') << "ALIAS EXPRESSION\n"; - alias_expression->dumpTreeImpl(buffer, state, indent + 4); + buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION\n"; + expression->dumpTreeImpl(buffer, state, indent + 4); } } bool ColumnNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); - if (column != rhs_typed.column) + if (column != rhs_typed.column || column_name_qualification != rhs_typed.column_name_qualification) return false; auto source_ptr = column_source.lock(); @@ -93,6 +112,9 @@ void ColumnNode::updateTreeHashImpl(HashState & hash_state) const hash_state.update(column_type_name.size()); hash_state.update(column_type_name); + hash_state.update(column_name_qualification.size()); + hash_state.update(column_name_qualification); + auto column_source_ptr = column_source.lock(); if (column_source_ptr) column_source_ptr->updateTreeHashImpl(hash_state); @@ -100,7 +122,7 @@ void ColumnNode::updateTreeHashImpl(HashState & hash_state) const QueryTreeNodePtr ColumnNode::cloneImpl() const { - return std::make_shared(column, column_source); + return std::make_shared(column, column_name_qualification, column_source); } void ColumnNode::getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update) @@ -115,6 +137,12 @@ void ColumnNode::getPointersToUpdateAfterClone(QueryTreePointersToUpdate & point ASTPtr ColumnNode::toASTImpl() const { + if (!column_name_qualification.empty()) + { + std::vector parts = {column_name_qualification, column.name}; + return std::make_shared(std::move(parts)); + } + return std::make_shared(column.name); } diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index 5a27e23c5cb..4784064d0d6 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -12,7 +12,14 @@ namespace DB * Column can be table expression, lambda, subquery. * Column source must be valid during column node lifetime. * - * For table ALIAS columns. Column node must contain ALIAS expression. + * For table ALIAS columns. Column node must contain expression. + * For ARRAY JOIN join expression column. Column node must contain expression. + * + * Additionaly column must be initialized with column name qualification if there are multiple + * unqualified columns with same name in query scope. + * Example: SELECT a.id, b.id FROM test_table_join_1 AS a, test_table_join_1. + * Both columns a.id and b.id have same unqualified name id. And additionally must be initialized + * with qualification a and b. * * During query analysis pass identifier node is resolved into column. See IdentifierNode.h. * @@ -34,8 +41,14 @@ public: /// Construct column node with column name, type and column source weak pointer. ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_); - /// Construct ALIAS column node with column name, type, column expression and column source weak pointer. - ColumnNode(NameAndTypePair column_, QueryTreeNodePtr alias_expression_node_, QueryTreeNodeWeakPtr column_source_); + /// Construct expression column node with column name, type, column expression and column source weak pointer. + ColumnNode(NameAndTypePair column_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_); + + /// Construct column node with column name, type, column name qualification and column source weak pointer. + ColumnNode(NameAndTypePair column_, String column_name_qualification_, QueryTreeNodeWeakPtr column_source_); + + /// Construct expression column node with column name, type, column name qualification column expression and column source weak pointer. + ColumnNode(NameAndTypePair column_, String column_name_qualification_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_); /// Get column const NameAndTypePair & getColumn() const @@ -49,25 +62,35 @@ public: return column.name; } + bool hasColumnNameQualfication() const + { + return !column_name_qualification.empty(); + } + + const String & getColumnQualification() const + { + return column_name_qualification; + } + /// Get column type const DataTypePtr & getColumnType() const { return column.type; } - bool hasAliasExpression() const + bool hasExpression() const { - return children[alias_expression_child_index] != nullptr; + return children[expression_child_index] != nullptr; } - const QueryTreeNodePtr & getAliasExpression() const + const QueryTreeNodePtr & getExpression() const { - return children[alias_expression_child_index]; + return children[expression_child_index]; } - QueryTreeNodePtr & getAliasExpression() + QueryTreeNodePtr & getExpression() { - return children[alias_expression_child_index]; + return children[expression_child_index]; } /** Get column source. @@ -111,9 +134,11 @@ protected: private: NameAndTypePair column; + String column_name_qualification; QueryTreeNodeWeakPtr column_source; - static constexpr size_t alias_expression_child_index = 0; + static constexpr size_t expression_child_index = 0; + static constexpr size_t children_size = expression_child_index + 1; }; } diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index b947d43ec0f..051976db228 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -33,6 +33,8 @@ const char * toString(QueryTreeNodeType type) case QueryTreeNodeType::TABLE: return "TABLE"; case QueryTreeNodeType::TABLE_FUNCTION: return "TABLE_FUNCTION"; case QueryTreeNodeType::QUERY: return "QUERY"; + case QueryTreeNodeType::ARRAY_JOIN: return "ARRAY_JOIN"; + case QueryTreeNodeType::JOIN: return "JOIN"; } } diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 75c3af87c3a..b45ec003bdc 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -38,6 +38,8 @@ enum class QueryTreeNodeType TABLE, TABLE_FUNCTION, QUERY, + ARRAY_JOIN, + JOIN }; /// Convert query tree node type to string diff --git a/src/Analyzer/JoinNode.cpp b/src/Analyzer/JoinNode.cpp new file mode 100644 index 00000000000..5227b82c119 --- /dev/null +++ b/src/Analyzer/JoinNode.cpp @@ -0,0 +1,159 @@ +#include +#include + +#include +#include +#include + +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + +JoinNode::JoinNode(QueryTreeNodePtr left_table_expression_, + QueryTreeNodePtr right_table_expression_, + QueryTreeNodePtr join_expression_, + JoinLocality locality_, + JoinStrictness strictness_, + JoinKind kind_) + : locality(locality_) + , strictness(strictness_) + , kind(kind_) +{ + children.resize(children_size); + children[left_table_expression_child_index] = std::move(left_table_expression_); + children[right_table_expression_child_index] = std::move(right_table_expression_); + children[join_expression_child_index] = std::move(join_expression_); +} + +JoinNode::JoinNode(QueryTreeNodePtr left_table_expression_, + QueryTreeNodePtr right_table_expression_, + QueryTreeNodes using_identifiers, + JoinLocality locality_, + JoinStrictness strictness_, + JoinKind kind_) + : locality(locality_) + , strictness(strictness_) + , kind(kind_) +{ + children.resize(children_size); + children[left_table_expression_child_index] = std::move(left_table_expression_); + children[right_table_expression_child_index] = std::move(right_table_expression_); + children[join_expression_child_index] = std::make_shared(std::move(using_identifiers)); +} + +JoinNode::JoinNode(JoinLocality locality_, + JoinStrictness strictness_, + JoinKind kind_) + : locality(locality_) + , strictness(strictness_) + , kind(kind_) +{} + +void JoinNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const +{ + buffer << std::string(indent, ' ') << "JOIN id: " << format_state.getNodeId(this); + + if (locality != JoinLocality::Unspecified) + buffer << ", locality: " << toString(locality); + + if (strictness != JoinStrictness::Unspecified) + buffer << ", strictness: " << toString(strictness); + + buffer << ", kind: " << toString(kind); + + buffer << '\n' << std::string(indent + 2, ' ') << "LEFT TABLE EXPRESSION\n"; + getLeftTableExpression()->dumpTreeImpl(buffer, format_state, indent + 4); + + buffer << '\n' << std::string(indent + 2, ' ') << "RIGHT TABLE EXPRESSSION\n"; + getRightTableExpression()->dumpTreeImpl(buffer, format_state, indent + 4); + + if (getJoinExpression()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "JOIN EXPRESSSION\n"; + getJoinExpression()->dumpTreeImpl(buffer, format_state, indent + 4); + } +} + +bool JoinNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + return locality == rhs_typed.locality && strictness == rhs_typed.strictness && kind == rhs_typed.kind; +} + +void JoinNode::updateTreeHashImpl(HashState & state) const +{ + state.update(locality); + state.update(strictness); + state.update(kind); +} + +ASTPtr JoinNode::toASTTableJoin() const +{ + auto join_ast = std::make_shared(); + join_ast->locality = locality; + join_ast->strictness = strictness; + join_ast->kind = kind; + + if (children[join_expression_child_index]) + { + auto join_expression_ast = children[join_expression_child_index]->toAST(); + + if (children[join_expression_child_index]->getNodeType() == QueryTreeNodeType::LIST) + join_ast->using_expression_list = std::move(join_expression_ast); + else + join_ast->on_expression = std::move(join_expression_ast); + } + + return join_ast; +} + +ASTPtr JoinNode::toASTImpl() const +{ + ASTPtr tables_in_select_query_ast = std::make_shared(); + + addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[left_table_expression_child_index]); + + size_t join_table_index = tables_in_select_query_ast->children.size(); + + auto join_ast = std::make_shared(); + join_ast->locality = locality; + join_ast->strictness = strictness; + join_ast->kind = kind; + + if (children[join_expression_child_index]) + { + auto join_expression_ast = children[join_expression_child_index]->toAST(); + + if (children[join_expression_child_index]->getNodeType() == QueryTreeNodeType::LIST) + join_ast->using_expression_list = std::move(join_expression_ast); + else + join_ast->on_expression = std::move(join_expression_ast); + } + + addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[right_table_expression_child_index]); + + auto & table_element = tables_in_select_query_ast->children.at(join_table_index)->as(); + table_element.children.push_back(std::move(join_ast)); + table_element.table_join = table_element.children.back(); + + return tables_in_select_query_ast; +} + +QueryTreeNodePtr JoinNode::cloneImpl() const +{ + JoinNodePtr result_join_node(new JoinNode(locality, strictness, kind)); + return result_join_node; +} + +} diff --git a/src/Analyzer/JoinNode.h b/src/Analyzer/JoinNode.h new file mode 100644 index 00000000000..3bde9a25b84 --- /dev/null +++ b/src/Analyzer/JoinNode.h @@ -0,0 +1,148 @@ +#pragma once + +#include + +#include +#include +#include + +#include +#include + +#include + +namespace DB +{ + +/** Array join node represents array join in query tree. + * Example: SELECT id FROM test_table ARRAY JOIN [1, 2, 3]. + */ +class JoinNode; +using JoinNodePtr = std::shared_ptr; + +class JoinNode final : public IQueryTreeNode +{ +public: + /** Construct join node with left table expression, right table expression and join expression. + * Example: SELECT id FROM test_table_1 INNER JOIN test_table_2 ON expression. + * + * test_table_1 - left table expression. + * test_table_2 - right table expression. + * join_expression - join_expression; + */ + JoinNode(QueryTreeNodePtr left_table_expression_, + QueryTreeNodePtr right_table_expression_, + QueryTreeNodePtr join_expression_, + JoinLocality locality_, + JoinStrictness strictness_, + JoinKind kind_); + + /** Construct join node with left table expression, right table expression and using identifiers. + * Example: SELECT id FROM test_table_1 INNER JOIN test_table_2 USING (using_identifier, ...). + * test_table_1 - left table expression. + * test_table_2 - right table expression. + * (using_identifier, ...) - using identifiers. + */ + JoinNode(QueryTreeNodePtr left_table_expression_, + QueryTreeNodePtr right_table_expression_, + QueryTreeNodes using_identifiers, + JoinLocality locality_, + JoinStrictness strictness_, + JoinKind kind_); + + /// Get left table expression + const QueryTreeNodePtr & getLeftTableExpression() const + { + return children[left_table_expression_child_index]; + } + + /// Get left table expression + QueryTreeNodePtr & getLeftTableExpression() + { + return children[left_table_expression_child_index]; + } + + /// Get right table expression + const QueryTreeNodePtr & getRightTableExpression() const + { + return children[right_table_expression_child_index]; + } + + /// Get right table expression + QueryTreeNodePtr & getRightTableExpression() + { + return children[right_table_expression_child_index]; + } + + /// Get join expression + const QueryTreeNodePtr & getJoinExpression() const + { + return children[join_expression_child_index]; + } + + /// Get join expression + QueryTreeNodePtr & getJoinExpression() + { + return children[join_expression_child_index]; + } + + bool isUsingJoinExpression() const + { + return getJoinExpression() && getJoinExpression()->getNodeType() == QueryTreeNodeType::LIST; + } + + bool isOnJoinExpression() const + { + return getJoinExpression() && getJoinExpression()->getNodeType() != QueryTreeNodeType::LIST; + } + + JoinLocality getLocality() const + { + return locality; + } + + JoinStrictness getStrictness() const + { + return strictness; + } + + JoinKind getKind() const + { + return kind; + } + + ASTPtr toASTTableJoin() const; + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::JOIN; + } + + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + + void updateTreeHashImpl(HashState & state) const override; + +protected: + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + JoinNode(JoinLocality locality_, + JoinStrictness strictness_, + JoinKind kind_); + + JoinLocality locality = JoinLocality::Unspecified; + JoinStrictness strictness = JoinStrictness::Unspecified; + JoinKind kind = JoinKind::Inner; + + static constexpr size_t left_table_expression_child_index = 0; + static constexpr size_t right_table_expression_child_index = 1; + static constexpr size_t join_expression_child_index = 2; + static constexpr size_t children_size = join_expression_child_index + 1; +}; + +} + diff --git a/src/Analyzer/ListNode.cpp b/src/Analyzer/ListNode.cpp index ffd72a3990e..59b42d384de 100644 --- a/src/Analyzer/ListNode.cpp +++ b/src/Analyzer/ListNode.cpp @@ -11,6 +11,11 @@ namespace DB { +ListNode::ListNode(QueryTreeNodes nodes) +{ + children = std::move(nodes); +} + void ListNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "LIST id: " << format_state.getNodeId(this); diff --git a/src/Analyzer/ListNode.h b/src/Analyzer/ListNode.h index 413dc964612..f93043144e0 100644 --- a/src/Analyzer/ListNode.h +++ b/src/Analyzer/ListNode.h @@ -16,6 +16,12 @@ using ListNodePtr = std::shared_ptr; class ListNode final : public IQueryTreeNode { public: + /// Initialize list node with empty nodes + ListNode() = default; + + /// Initialize list node with nodes + explicit ListNode(QueryTreeNodes nodes); + /// Get list nodes const QueryTreeNodes & getNodes() const { diff --git a/src/Analyzer/MatcherNode.cpp b/src/Analyzer/MatcherNode.cpp index 3aa70557987..6393193ec01 100644 --- a/src/Analyzer/MatcherNode.cpp +++ b/src/Analyzer/MatcherNode.cpp @@ -178,6 +178,8 @@ String MatcherNode::getName() const } buffer << ')'; + /// TODO: Transformers + return buffer.str(); } diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index d20e8598e03..b31457f57c0 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -42,6 +42,8 @@ #include #include #include +#include +#include #include #include @@ -82,6 +84,7 @@ namespace ErrorCodes extern const int UNKNOWN_DATABASE; extern const int UNKNOWN_TABLE; extern const int INCORRECT_ELEMENT_OF_SET; + extern const int TYPE_MISMATCH; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -129,7 +132,7 @@ namespace ErrorCodes * Resolution starts with current scope. * 1. Try to resolve identifier from expression scope arguments. Lambda expression arguments are greatest priority. * 2. Try to resolve identifier from aliases. - * 3. Try to resolve identifier from tables if scope is query, or if there are registered table columns in scope. + * 3. Try to resolve identifier from join tree if scope is query, or if there are registered table columns in scope. * Steps 2 and 3 can be changed using prefer_column_name_to_alias setting. * 4. If it is table lookup, try to resolve identifier from CTE. * If identifier could not be resolved in current scope, resolution must be continued in parent scopes. @@ -151,7 +154,7 @@ namespace ErrorCodes * CREATE TABLE test_table (id UInt64, compound_value Tuple(value UInt64)) ENGINE=TinyLog; * SELECT compound_value.value, 1 AS compound_value FROM test_table; * Identifier first part compound_value bound to entity with alias compound_value, but nested identifier part cannot be resolved from entity, - * lookup should not be continued, and exception must be throwed because if lookup continues that way identifier can be resolved from tables. + * lookup should not be continued, and exception must be throwed because if lookup continues that way identifier can be resolved from join tree. * * TODO: This was not supported properly before analyzer because nested identifier could not be resolved from alias. * @@ -162,16 +165,23 @@ namespace ErrorCodes * lookup should not be continued, and exception must be throwed because if lookup continues identifier can be resolved from parent scope. * * TODO: Add expression name into query tree node. Example: SELECT plus(1, 1). Result: SELECT 2. Expression name of constant node should be 2. + * TODO: Disable identifier cache during join tree resolve * TODO: Update exception messages + * TODO: JOIN TREE subquery constant columns + * TODO: Column name qualifications * TODO: Table identifiers with optional UUID. * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. * TODO: UNION - * TODO: JOIN, ARRAY JOIN + * TODO: JOIN support LEFT, RIGHT with nullable. JOIN support USING. JOIN support SELF JOIN with MergeTree. JOIN support matchers. * TODO: WINDOW functions + * TODO: Table expression modifiers final, sample_size, sample_offset */ +namespace +{ + /// Identifier lookup context enum class IdentifierLookupContext : uint8_t { @@ -234,7 +244,7 @@ inline bool operator==(const IdentifierLookup & lhs, const IdentifierLookup & rh return lhs.identifier.getFullName() == rhs.identifier.getFullName() && lhs.lookup_context == rhs.lookup_context; } -inline bool operator!=(const IdentifierLookup & lhs, const IdentifierLookup & rhs) +[[maybe_unused]] inline bool operator!=(const IdentifierLookup & lhs, const IdentifierLookup & rhs) { return !(lhs == rhs); } @@ -252,7 +262,7 @@ enum class IdentifierResolvePlace : UInt8 NONE = 0, EXPRESSION_ARGUMENTS, ALIASES, - TABLES, + JOIN_TREE, /// Valid only for table lookup CTE, /// Valid only for table lookup @@ -266,7 +276,7 @@ static const char * toString(IdentifierResolvePlace resolved_identifier_place) case IdentifierResolvePlace::NONE: return "NONE"; case IdentifierResolvePlace::EXPRESSION_ARGUMENTS: return "EXPRESSION_ARGUMENTS"; case IdentifierResolvePlace::ALIASES: return "ALIASES"; - case IdentifierResolvePlace::TABLES: return "TABLES"; + case IdentifierResolvePlace::JOIN_TREE: return "JOIN_TREE"; case IdentifierResolvePlace::CTE: return "CTE"; case IdentifierResolvePlace::DATABASE_CATALOG: return "DATABASE_CATALOG"; } @@ -280,32 +290,32 @@ struct IdentifierResolveResult IdentifierResolvePlace resolve_place = IdentifierResolvePlace::NONE; bool resolved_from_parent_scopes = false; - bool isResolved() const + [[maybe_unused]] bool isResolved() const { return resolve_place != IdentifierResolvePlace::NONE; } - bool isResolvedFromParentScopes() const + [[maybe_unused]] bool isResolvedFromParentScopes() const { return resolved_from_parent_scopes; } - bool isResolvedFromExpressionArguments() const + [[maybe_unused]] bool isResolvedFromExpressionArguments() const { return resolve_place == IdentifierResolvePlace::EXPRESSION_ARGUMENTS; } - bool isResolvedFromAliases() const + [[maybe_unused]] bool isResolvedFromAliases() const { return resolve_place == IdentifierResolvePlace::ALIASES; } - bool isResolvedFromTables() const + [[maybe_unused]] bool isResolvedFromJoinTree() const { - return resolve_place == IdentifierResolvePlace::TABLES; + return resolve_place == IdentifierResolvePlace::JOIN_TREE; } - bool isResolvedFromCTEs() const + [[maybe_unused]] bool isResolvedFromCTEs() const { return resolve_place == IdentifierResolvePlace::CTE; } @@ -320,6 +330,14 @@ struct IdentifierResolveResult buffer << resolved_identifier->formatASTForErrorMessage() << " place " << toString(resolve_place) << " resolved from parent scopes " << resolved_from_parent_scopes; } + + [[maybe_unused]] String dump() const + { + WriteBufferFromOwnString buffer; + dump(buffer); + + return buffer.str(); + } }; struct IdentifierResolveSettings @@ -327,6 +345,9 @@ struct IdentifierResolveSettings /// Allow to check parent scopes during identifier resolution bool allow_to_check_parent_scopes = true; + /// Allow to check join tree during identifier resolution + bool allow_to_check_join_tree = true; + /// Allow to check CTEs during table identifer resolution bool allow_to_check_cte = true; @@ -344,14 +365,14 @@ struct TableExpressionColumns return column_identifier_first_parts.find(std::string(identifier.at(0))) != column_identifier_first_parts.end(); } - void dump(WriteBuffer & buffer) + [[maybe_unused]] void dump(WriteBuffer & buffer) { buffer << "Columns size " << column_name_to_column_node.size() << '\n'; for (auto & [column_name, column_node] : column_name_to_column_node) { buffer << "Column name " << column_name << " column node " << column_node->formatASTForErrorMessage(); - buffer << " is alias " << column_node->hasAliasExpression() << '\n'; + buffer << " is alias " << column_node->hasExpression() << '\n'; } } }; @@ -407,7 +428,7 @@ public: return alias_name_to_expressions_size.find(alias) != alias_name_to_expressions_size.end(); } - size_t size() const + [[maybe_unused]] size_t size() const { return expressions.size(); } @@ -475,6 +496,9 @@ struct IdentifierResolveScope /// Current scope expression in resolve process stack ExpressionsStack expressions_in_resolve_process_stack; + /// Table expressions in resolve process + std::unordered_set table_expressions_in_resolve_process; + /// Current scope expression std::unordered_set non_cached_identifier_lookups_during_expression_resolve; @@ -482,7 +506,7 @@ struct IdentifierResolveScope bool allow_to_check_parent_scopes = true; /// Dump identifier resolve scope - void dump(WriteBuffer & buffer) + [[maybe_unused]] void dump(WriteBuffer & buffer) { buffer << "Scope node " << scope_node->formatASTForErrorMessage() << '\n'; buffer << "Identifier lookup to result " << identifier_lookup_to_result.size() << '\n'; @@ -490,6 +514,7 @@ struct IdentifierResolveScope { buffer << "Identifier " << identifier.dump() << " resolve result "; result.dump(buffer); + buffer << '\n'; } buffer << "Expression argument name to node " << expression_argument_name_to_node.size() << '\n'; @@ -524,6 +549,162 @@ struct IdentifierResolveScope } }; + +/** Visitor that extracts expression and function aliases from node and initialize scope tables with it. + * Does not go into child lambdas and queries. + * + * Important: + * Identifier nodes with aliases are added both in alias to expression and alias to function map. + * + * These is necessary because identifier with alias can give alias name to any query tree node. + * + * Example: + * WITH (x -> x + 1) AS id, id AS value SELECT value(1); + * In this example id as value is identifier node that has alias, during scope initialization we cannot derive + * that id is actually lambda or expression. + * + * There are no easy solution here, without trying to make full featured expression resolution at this stage. + * Example: + * WITH (x -> x + 1) AS id, id AS id_1, id_1 AS id_2 SELECT id_2(1); + * + * It is client responsibility after resolving identifier node with alias, make following actions: + * 1. If identifier node was resolved in function scope, remove alias from scope expression map. + * 2. If identifier node was resolved in expression scope, remove alias from scope function map. + * + * That way we separate alias map initialization and expressions resolution. + */ +class QueryExpressionsAliasVisitorMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + IdentifierResolveScope & scope; + }; + + static void visit(QueryTreeNodePtr & node, Data & data) + { + updateAliasesIfNeeded(data, node, false); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child, Data & data) + { + if (auto * lambda_node = child->as()) + { + updateAliasesIfNeeded(data, child, true); + return false; + } + else if (auto * query_tree_node = child->as()) + { + if (query_tree_node->isCTE()) + return false; + + updateAliasesIfNeeded(data, child, false); + return false; + } + + return true; + } +private: + static void updateAliasesIfNeeded(Data & data, const QueryTreeNodePtr & node, bool function_node) + { + if (!node->hasAlias()) + return; + + const auto & alias = node->getAlias(); + + if (function_node) + { + if (data.scope.alias_name_to_expression_node.contains(alias)) + data.scope.nodes_with_duplicated_aliases.insert(node); + + auto [_, inserted] = data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); + if (!inserted) + data.scope.nodes_with_duplicated_aliases.insert(node); + + return; + } + + if (data.scope.alias_name_to_lambda_node.contains(alias)) + data.scope.nodes_with_duplicated_aliases.insert(node); + + auto [_, inserted] = data.scope.alias_name_to_expression_node.insert(std::make_pair(alias, node)); + if (!inserted) + data.scope.nodes_with_duplicated_aliases.insert(node); + + /// If node is identifier put it also in scope alias name to lambda node map + if (node->getNodeType() == QueryTreeNodeType::IDENTIFIER) + data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); + } +}; + +using QueryExpressionsAliasVisitor = QueryExpressionsAliasVisitorMatcher::Visitor; + +class TableExpressionsAliasVisitorMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + IdentifierResolveScope & scope; + }; + + static void visit(QueryTreeNodePtr & node, Data & data) + { + updateAliasesIfNeeded(data, node); + } + + static bool needChildVisit(const QueryTreeNodePtr & node, const QueryTreeNodePtr & child) + { + auto node_type = node->getNodeType(); + + switch (node_type) + { + case QueryTreeNodeType::ARRAY_JOIN: + { + const auto & array_join_node = node->as(); + return child.get() == array_join_node.getTableExpression().get(); + } + case QueryTreeNodeType::JOIN: + { + const auto & join_node = node->as(); + return child.get() == join_node.getLeftTableExpression().get() || child.get() == join_node.getRightTableExpression().get(); + } + default: + { + break; + } + } + + return false; + } + +private: + static void updateAliasesIfNeeded(Data & data, const QueryTreeNodePtr & node) + { + if (!node->hasAlias()) + return; + + const auto & node_alias = node->getAlias(); + auto [_, inserted] = data.scope.alias_name_to_table_expression_node.emplace(node_alias, node); + if (!inserted) + throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, + "Multiple table expressions with same alias {}. In scope {}", + node_alias, + data.scope.scope_node->formatASTForErrorMessage()); + } +}; + +using TableExpressionsAliasVisitor = TableExpressionsAliasVisitorMatcher::Visitor; + +struct StorageLockAndSnapshot +{ + TableLockHolder lock; + StorageSnapshotPtr snapshot; +}; + class QueryAnalyzer { public: @@ -565,8 +746,6 @@ private: static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path); - static NamesAndTypesList getQueryNodeColumns(const QueryTreeNodePtr & query_node); - QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); void evaluateScalarSubquery(QueryTreeNodePtr & query_tree_node); @@ -581,7 +760,11 @@ private: QueryTreeNodePtr tryResolveIdentifierFromTableColumns(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); - QueryTreeNodePtr tryResolveIdentifierFromTables(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); + QueryTreeNodePtr tryResolveIdentifierFromTable(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + + QueryTreeNodePtr tryResolveIdentifierFromJoinTreeNode(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope); + + QueryTreeNodePtr tryResolveIdentifierFromJoinTree(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); IdentifierResolveResult tryResolveIdentifierInParentScopes(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); @@ -599,9 +782,11 @@ private: void resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); - void initializeQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope); + void initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope); - void resolveQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope); + void initializeTableExpressionColumns(QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + + void resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope, QueryExpressionsAliasVisitor & expressions_visitor); void resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope); @@ -611,11 +796,11 @@ private: /// Lambdas that are currently in resolve process std::unordered_set lambdas_in_resolve_process; - /// Query nodes where FROM section is in resolve process - std::unordered_set query_node_from_section_in_resolve_process; - /// Table expression node to columns cache std::unordered_map table_expression_node_to_columns; + + /// Array join expressions counter + size_t array_join_expressions_counter = 0; }; /// Utility functions implementation @@ -639,33 +824,6 @@ QueryTreeNodePtr QueryAnalyzer::wrapExpressionNodeInTupleElement(QueryTreeNodePt return expression_node; } -/** Get query node columns using projection section - */ -NamesAndTypesList QueryAnalyzer::getQueryNodeColumns(const QueryTreeNodePtr & query_node) -{ - const auto & query_node_typed = query_node->as(); - - NamesAndTypes query_columns; - - const auto & projection_nodes = query_node_typed.getProjection(); - query_columns.reserve(projection_nodes.getNodes().size()); - - for (const auto & projection_node : projection_nodes.getNodes()) - { - auto column_type = projection_node->getResultType(); - std::string column_name; - - if (projection_node->hasAlias()) - column_name = projection_node->getAlias(); - else - column_name = projection_node->getName(); - - query_columns.emplace_back(column_name, column_type); - } - - return {query_columns.begin(), query_columns.end()}; -} - /** Try to get lambda node from sql user defined functions if sql user defined function with function name exists. * Returns lambda node if function exists, nullptr otherwise. */ @@ -818,12 +976,14 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con if (!database) throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exists", database_name); - auto table = database->tryGetTable(table_name, context); + auto storage = database->tryGetTable(table_name, context); - if (!table) + if (!storage) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exists", table_name); - return std::make_shared(std::move(table), context); + auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); + auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); + return std::make_shared(std::move(storage), storage_lock, storage_snapshot); } /** Resolve identifier from expression arguments. @@ -875,97 +1035,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(cons return it->second; } -/** Visitor that extracts expression and function aliases from node and initialize scope tables with it. - * Does not go into child lambdas and queries. - * - * Important: - * Identifier nodes with aliases are added both in alias to expression and alias to function map. - * - * These is necessary because identifier with alias can give alias name to any query tree node. - * - * Example: - * WITH (x -> x + 1) AS id, id AS value SELECT value(1); - * In this example id as value is identifier node that has alias, during scope initialization we cannot derive - * that id is actually lambda or expression. - * - * There are no easy solution here, without trying to make full featured expression resolution at this stage. - * Example: - * WITH (x -> x + 1) AS id, id AS id_1, id_1 AS id_2 SELECT id_2(1); - * - * It is client responsibility after resolving identifier node with alias, make following actions: - * 1. If identifier node was resolved in function scope, remove alias from scope expression map. - * 2. If identifier node was resolved in expression scope, remove alias from scope function map. - * - * That way we separate alias map initialization and expressions resolution. - */ -class ScopeAliasVisitorMatcher -{ -public: - using Visitor = InDepthQueryTreeVisitor; - - struct Data - { - IdentifierResolveScope & scope; - }; - - static void visit(QueryTreeNodePtr & node, Data & data) - { - updateAliasesIfNeeded(data, node, false); - } - - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child, Data & data) - { - if (auto * lambda_node = child->as()) - { - updateAliasesIfNeeded(data, child, true); - return false; - } - else if (auto * query_tree_node = child->as()) - { - if (query_tree_node->isCTE()) - return false; - - updateAliasesIfNeeded(data, child, false); - return false; - } - - return !(child->as()); - } -private: - static void updateAliasesIfNeeded(Data & data, const QueryTreeNodePtr & node, bool function_node) - { - if (!node->hasAlias()) - return; - - const auto & alias = node->getAlias(); - - if (function_node) - { - if (data.scope.alias_name_to_expression_node.contains(alias)) - data.scope.nodes_with_duplicated_aliases.insert(node); - - auto [_, inserted] = data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); - if (!inserted) - data.scope.nodes_with_duplicated_aliases.insert(node); - - return; - } - - if (data.scope.alias_name_to_lambda_node.contains(alias)) - data.scope.nodes_with_duplicated_aliases.insert(node); - - auto [_, inserted] = data.scope.alias_name_to_expression_node.insert(std::make_pair(alias, node)); - if (!inserted) - data.scope.nodes_with_duplicated_aliases.insert(node); - - /// If node is identifier put it also in scope alias name to lambda node map - if (node->getNodeType() == QueryTreeNodeType::IDENTIFIER) - data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); - } -}; - -using ScopeAliasVisitor = ScopeAliasVisitorMatcher::Visitor; - /** Resolve identifier from scope aliases. * * Resolve strategy: @@ -1000,7 +1069,7 @@ using ScopeAliasVisitor = ScopeAliasVisitorMatcher::Visitor; * Example: SELECT value.a, cast('(1)', 'Tuple(a UInt64)') AS value; * * Special case for IdentifierNode, if node is identifier depending on lookup context we need to erase entry from expression or lambda map. - * Check ScopeAliasVisitorMatcher documentation. + * Check QueryExpressionsAliasVisitor documentation. * * Special case for QueryNode, if lookup context is expression, evaluate it as scalar subquery. * @@ -1062,22 +1131,15 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier auto lookup_result = tryResolveIdentifier(IdentifierLookup{alias_identifier->getIdentifier(), identifier_lookup.lookup_context}, scope, identifier_resolve_settings); it->second = lookup_result.resolved_identifier; - if (!it->second) - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Unknown {} identifier {}. In scope {}", - toStringLowercase(identifier_lookup.lookup_context), - identifier_lookup.identifier.getFullName(), - scope.scope_node->formatASTForErrorMessage()); - /** During collection of aliases if node is identifier and has alias, we cannot say if it is - * column or function node. Check ScopeAliasVisitor documentation for clarification. + * column or function node. Check QueryExpressionsAliasVisitor documentation for clarification. * * If we resolved identifier node as expression, we must remove identifier node alias from * function alias map. * If we resolved identifier node as function, we must remove identifier node alias from * expression alias map. */ - if (identifier_lookup.isExpressionLookup()) + if (identifier_lookup.isExpressionLookup() && it->second) scope.alias_name_to_lambda_node.erase(identifier_bind_part); else if (identifier_lookup.isFunctionLookup() && it->second) scope.alias_name_to_expression_node.erase(identifier_bind_part); @@ -1141,8 +1203,8 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableColumns(const Ident return {}; } - if (it->second->hasAliasExpression()) - resolveExpressionNode(it->second->getAliasExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + if (it->second->hasExpression()) + resolveExpressionNode(it->second->getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); QueryTreeNodePtr result = it->second; @@ -1160,47 +1222,17 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableColumns(const Ident return result; } -/** Resolve identifier from scope tables. - * - * 1. If identifier is in function lookup context return nullptr. - * 2. Try to resolve identifier from table columns. - * 3. If there is no FROM section return nullptr. - * 4. If identifier is in table lookup context, check if it has 1 or 2 parts, otherwise throw exception. - * If identifer has 2 parts try to match it with database_name and table_name. - * If identifier has 1 part try to match it with table_name, then try to match it with table alias. - * 5. If identifier is in expression lookup context, we first need to bind identifier to some table column using identifier first part. - * Start with identifier first part, if it match some column name in table try to get column with full identifier name. - * TODO: Need to check if it is okay to throw exception if compound identifier first part bind to column but column is not valid. - */ -QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) { - if (identifier_lookup.isFunctionLookup()) - return {}; - - /// Try to resolve identifier from table columns - - if (auto resolved_identifier = tryResolveIdentifierFromTableColumns(identifier_lookup, scope)) - return resolved_identifier; - - auto * query_scope_node = scope.scope_node->as(); - if (!query_scope_node || !query_scope_node->getFrom()) - return {}; - - auto from_node = query_scope_node->getFrom(); - - auto * table_node = from_node->as(); - auto * query_node = from_node->as(); - auto * table_function_node = from_node->as(); - - /** Edge case scenario when subquery in FROM node try to resolve identifier from parent scopes, when FROM is not resolved. - * SELECT subquery.b AS value FROM (SELECT value, 1 AS b) AS subquery; - * TODO: This can be supported - */ - if (query_node_from_section_in_resolve_process.contains(query_scope_node)) - return {}; + auto * table_node = table_expression_node->as(); + auto * query_node = table_expression_node->as(); + auto * table_function_node = table_expression_node->as(); if (!table_node && !table_function_node && !query_node) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "FROM does not contain table, table function or query node. Actual {}", from_node->formatASTForErrorMessage()); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Unexpected table expression. Expected table, table function or query node. Actual {}. In scope {}", + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); const auto & identifier = identifier_lookup.identifier; const auto & path_start = identifier.getParts().front(); @@ -1240,102 +1272,25 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL table_node->formatASTForErrorMessage()); if (parts_size == 1 && path_start == table_name) - return from_node; + return table_expression_node; else if (parts_size == 2 && path_start == database_name && identifier[1] == table_name) - return from_node; + return table_expression_node; else return {}; } - auto storage_columns_it = table_expression_node_to_columns.find(from_node.get()); + auto storage_columns_it = table_expression_node_to_columns.find(table_expression_node.get()); if (storage_columns_it == table_expression_node_to_columns.end()) - { - TableExpressionColumns storage_columns; - - if (table_node || table_function_node) - { - const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); - - auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); - const auto & columns_description = storage_snapshot->metadata->getColumns(); - - std::vector> alias_columns_to_resolve; - std::unordered_map column_name_to_column_node; - column_name_to_column_node.reserve(column_names_and_types.size()); - - for (const auto & column_name_and_type : column_names_and_types) - { - const auto & column_default = columns_description.getDefault(column_name_and_type.name); - - if (column_default && column_default->kind == ColumnDefaultKind::Alias) - { - auto argument_node = std::make_shared(column_name_and_type, buildQueryTree(column_default->expression, context), from_node); - column_name_to_column_node.emplace(column_name_and_type.name, argument_node); - alias_columns_to_resolve.emplace_back(column_name_and_type.name, argument_node); - } - else - { - auto argument_node = std::make_shared(column_name_and_type, from_node); - column_name_to_column_node.emplace(column_name_and_type.name, argument_node); - } - } - - for (auto & [alias_column_to_resolve_name, alias_column_to_resolve] : alias_columns_to_resolve) - { - /** Alias column could be potentially resolved during resolve of other ALIAS column. - * Example: CREATE TABLE test_table (id UInt64, alias_value_1 ALIAS id + alias_value_2, alias_value_2 ALIAS id + 5) ENGINE=TinyLog; - * - * During resolve of alias_value_1, alias_value_2 column will be resolved. - */ - alias_column_to_resolve = column_name_to_column_node[alias_column_to_resolve_name]; - - IdentifierResolveScope alias_column_resolve_scope(alias_column_to_resolve, nullptr /*parent_scope*/); - alias_column_resolve_scope.column_name_to_column_node = std::move(column_name_to_column_node); - - /// Initialize aliases in alias column scope - ScopeAliasVisitorMatcher::Data data{alias_column_resolve_scope}; - ScopeAliasVisitorMatcher::Visitor visitor(data); - - visitor.visit(alias_column_to_resolve->getAliasExpression()); - - resolveExpressionNode(alias_column_resolve_scope.scope_node, alias_column_resolve_scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - - column_name_to_column_node = std::move(alias_column_resolve_scope.column_name_to_column_node); - column_name_to_column_node[alias_column_to_resolve_name] = alias_column_to_resolve; - } - - storage_columns.column_name_to_column_node = std::move(column_name_to_column_node); - } - else if (query_node) - { - auto column_names_and_types = getQueryNodeColumns(from_node); - storage_columns.column_name_to_column_node.reserve(column_names_and_types.size()); - - for (const auto & column_name_and_type : column_names_and_types) - { - auto column_node = std::make_shared(column_name_and_type, from_node); - storage_columns.column_name_to_column_node.emplace(column_name_and_type.name, column_node); - } - } - - storage_columns.column_identifier_first_parts.reserve(storage_columns.column_name_to_column_node.size()); - - for (auto & [column_name, _] : storage_columns.column_name_to_column_node) - { - Identifier column_name_identifier(column_name); - storage_columns.column_identifier_first_parts.insert(column_name_identifier.at(0)); - } - - auto [inserted_it, _] = table_expression_node_to_columns.emplace(from_node.get(), std::move(storage_columns)); - storage_columns_it = inserted_it; - } + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Table expression {} columns must be initialized. In scope {}", + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); auto & storage_columns = storage_columns_it->second; - - auto resolve_identifier_from_storage_or_throw = [&](size_t drop_first_parts_size) -> QueryTreeNodePtr + auto resolve_identifier_from_storage_or_throw = [&](size_t identifier_column_qualifier_parts) -> QueryTreeNodePtr { auto identifier_view = IdentifierView(identifier); - identifier_view.popFirst(drop_first_parts_size); + identifier_view.popFirst(identifier_column_qualifier_parts); /** Compound identifier cannot be resolved directly from storage if storage is not table. * @@ -1346,7 +1301,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL * Here there is no column with name test_subquery.compound_expression.value, and additional wrap in tuple element is required. */ - QueryTreeNodePtr result_column; + ColumnNodePtr result_column; bool compound_identifier = identifier_view.getPartsSize() > 1; bool match_full_identifier = false; @@ -1364,7 +1319,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL result_column = it->second; } - if (!result_column) + if (!result_column || (!match_full_identifier && !compound_identifier)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Identifier {} cannot be resolved from {}{}. In scope {}", identifier.getFullName(), @@ -1372,6 +1327,17 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL table_expression_name.empty() ? "" : " with name " + table_expression_name, scope.scope_node->formatASTForErrorMessage()); + if (identifier_column_qualifier_parts > 0) + { + auto identifier_column_qualifier_view = IdentifierView(identifier); + identifier_column_qualifier_view.popLast(identifier.getPartsSize() - identifier_column_qualifier_parts); + + result_column = std::make_shared(result_column->getColumn(), + std::string(identifier_column_qualifier_view.getFullName()), + result_column->getExpression(), + table_expression_node); + } + QueryTreeNodePtr result_expression = result_column; if (!match_full_identifier && compound_identifier) @@ -1392,23 +1358,137 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL * 3. Try to bind identifier first parts to database name and table name, if true remove first two parts and try to get full identifier from table or throw exception. */ if (storage_columns.canBindIdentifier(IdentifierView(identifier))) - return resolve_identifier_from_storage_or_throw(0 /*drop_first_parts_size*/); + return resolve_identifier_from_storage_or_throw(0 /*identifier_column_qualifier_parts*/); if (identifier.getPartsSize() == 1) return {}; - if ((!table_name.empty() && path_start == table_name) || (from_node->hasAlias() && path_start == from_node->getAlias())) - return resolve_identifier_from_storage_or_throw(1 /*drop_first_parts_size*/); + if ((!table_name.empty() && path_start == table_name) || (table_expression_node->hasAlias() && path_start == table_expression_node->getAlias())) + return resolve_identifier_from_storage_or_throw(1 /*identifier_column_qualifier_parts*/); if (identifier.getPartsSize() == 2) return {}; if (!database_name.empty() && path_start == database_name && identifier[1] == table_name) - return resolve_identifier_from_storage_or_throw(2 /*drop_first_parts_size*/); + return resolve_identifier_from_storage_or_throw(2 /*identifier_column_qualifier_parts*/); return {}; } +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTreeNode(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope) +{ + auto join_tree_node_type = join_tree_node->getNodeType(); + + switch (join_tree_node_type) + { + case QueryTreeNodeType::JOIN: + { + const auto & from_join_node = join_tree_node->as(); + + auto left_resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_join_node.getLeftTableExpression(), scope); + auto right_resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_join_node.getRightTableExpression(), scope); + + if (left_resolved_identifier && right_resolved_identifier) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "JOIN {} ambigious identifier {}. In scope {}", + join_tree_node->formatASTForErrorMessage(), + identifier_lookup.identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + else if (left_resolved_identifier) + return left_resolved_identifier; + else if (right_resolved_identifier) + return right_resolved_identifier; + + return {}; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + const auto & from_array_join_node = join_tree_node->as(); + auto resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_array_join_node.getTableExpression(), scope); + + /** Special case when qualified or unqualified identifier point to array join expression. + * + * CREATE TABLE test_table (id UInt64, value String, value_array Array(UInt8)) ENGINE=TinyLog; + * SELECT id, value, value_array, test_table.value_array, default.test_table.value_array FROM test_table ARRAY JOIN value_array; + * + * value_array, test_table.value_array, default.test_table.value_array must be resolved into array join expression. + */ + if (resolved_identifier) + { + for (const auto & array_join_expression : from_array_join_node.getJoinExpressions().getNodes()) + { + auto * array_join_column_expression = array_join_expression->as(); + if (!array_join_column_expression) + continue; + + auto & array_join_column_inner_expression = array_join_column_expression->getExpression(); + + if (!array_join_column_inner_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "ARRAY JOIN column node must have inner expression. In scope {}", + scope.scope_node->formatASTForErrorMessage()); + + if (array_join_expression.get() == resolved_identifier.get() || array_join_expression->isEqual(*resolved_identifier)) + return array_join_expression; + } + } + + return resolved_identifier; + } + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + /** Edge case scenario when subquery in FROM node try to resolve identifier from parent scopes, when FROM is not resolved. + * SELECT subquery.b AS value FROM (SELECT value, 1 AS b) AS subquery; + * TODO: This can be supported + */ + if (scope.table_expressions_in_resolve_process.contains(join_tree_node.get())) + return {}; + + return tryResolveIdentifierFromTable(identifier_lookup, join_tree_node, scope); + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Scope FROM section expected table, table function, query, join or array join. Actual {}. In scope {}", + join_tree_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + } +} + +/** Resolve identifier from scope join tree. + * + * 1. If identifier is in function lookup context return nullptr. + * 2. Try to resolve identifier from table columns. + * 3. If there is no FROM section return nullptr. + * 4. If identifier is in table lookup context, check if it has 1 or 2 parts, otherwise throw exception. + * If identifer has 2 parts try to match it with database_name and table_name. + * If identifier has 1 part try to match it with table_name, then try to match it with table alias. + * 5. If identifier is in expression lookup context, we first need to bind identifier to some table column using identifier first part. + * Start with identifier first part, if it match some column name in table try to get column with full identifier name. + * TODO: Need to check if it is okay to throw exception if compound identifier first part bind to column but column is not valid. + */ +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTree(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +{ + if (identifier_lookup.isFunctionLookup()) + return {}; + + /// Try to resolve identifier from table columns + if (auto resolved_identifier = tryResolveIdentifierFromTableColumns(identifier_lookup, scope)) + return resolved_identifier; + + auto * query_scope_node = scope.scope_node->as(); + if (!query_scope_node || !query_scope_node->getFrom()) + return {}; + + const auto & join_tree_node = query_scope_node->getFrom(); + return tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_tree_node, scope); +} + /** Try resolve identifier in current scope parent scopes. * * TODO: If column is matched, throw exception that nested subqueries are not supported. @@ -1495,11 +1575,11 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const * 3. Try resolve identifier from expression arguments. * * If prefer_column_name_to_alias = true. - * 4. Try to resolve identifier from tables. + * 4. Try to resolve identifier from join tree. * 5. Try to resolve identifier from aliases. * Otherwise. * 4. Try to resolve identifier from aliases. - * 5. Try to resolve identifier from tables. + * 5. Try to resolve identifier from join tree. * * 6. If it is table identifier lookup try to lookup identifier in current scope CTEs. * @@ -1548,10 +1628,13 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook if (unlikely(prefer_column_name_to_alias)) { - resolve_result.resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, scope); + if (identifier_resolve_settings.allow_to_check_join_tree) + { + resolve_result.resolved_identifier = tryResolveIdentifierFromJoinTree(identifier_lookup, scope); - if (resolve_result.resolved_identifier) - resolve_result.resolve_place = IdentifierResolvePlace::TABLES; + if (resolve_result.resolved_identifier) + resolve_result.resolve_place = IdentifierResolvePlace::JOIN_TREE; + } if (!resolve_result.resolved_identifier) { @@ -1569,12 +1652,12 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook { resolve_result.resolve_place = IdentifierResolvePlace::ALIASES; } - else + else if (identifier_resolve_settings.allow_to_check_join_tree) { - resolve_result.resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, scope); + resolve_result.resolved_identifier = tryResolveIdentifierFromJoinTree(identifier_lookup, scope); if (resolve_result.resolved_identifier) - resolve_result.resolve_place = IdentifierResolvePlace::TABLES; + resolve_result.resolve_place = IdentifierResolvePlace::JOIN_TREE; } } } @@ -1703,7 +1786,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, if (table_expression_query_node) { - initial_matcher_columns = getQueryNodeColumns(table_expression_node); + initial_matcher_columns = table_expression_query_node->computeProjectionColumns(); } else { @@ -1753,7 +1836,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, if (from_query_node) { - initial_matcher_columns = getQueryNodeColumns(scope_query_node->getFrom()); + initial_matcher_columns = from_query_node->computeProjectionColumns(); } else if (from_table_node || from_table_function_node) { @@ -1860,6 +1943,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, } else if (auto * except_transformer = transformer->as()) { + auto node_name = node->getName(); if (except_transformer->isColumnMatching(node_name)) @@ -1873,6 +1957,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, } else if (auto * replace_transformer = transformer->as()) { + auto node_name = node->getName(); auto replace_expression = replace_transformer->findReplacementExpression(node_name); if (!replace_expression) @@ -2004,8 +2089,8 @@ void QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const Qu scope.scope_node->formatASTForErrorMessage()); /// Initialize aliases in lambda scope - ScopeAliasVisitorMatcher::Data data{scope}; - ScopeAliasVisitorMatcher::Visitor visitor(data); + QueryExpressionsAliasVisitor::Data data{scope}; + QueryExpressionsAliasVisitor visitor(data); visitor.visit(lambda.getExpression()); /** Replace lambda arguments with new arguments. @@ -2126,7 +2211,6 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc auto & function_argument = function_arguments[function_argument_index]; ColumnWithTypeAndName argument_column; - argument_column.name = function_argument->getName(); /** If function argument is lambda, save lambda argument index and initialize argument type as DataTypeFunction * where function argument types are initialized with empty array of lambda arguments size. @@ -2562,8 +2646,8 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes case QueryTreeNodeType::COLUMN: { auto & column_function_node = node->as(); - if (column_function_node.hasAliasExpression()) - resolveExpressionNode(column_function_node.getAliasExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + if (column_function_node.hasExpression()) + resolveExpressionNode(column_function_node.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); break; } case QueryTreeNodeType::FUNCTION: @@ -2610,6 +2694,20 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes break; } + case QueryTreeNodeType::ARRAY_JOIN: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Array join is not allowed {} in expression. In scope {}", + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + case QueryTreeNodeType::JOIN: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Join is not allowed {} in expression. In scope {}", + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } } /** Update aliases after expression node was resolved. @@ -2673,133 +2771,400 @@ void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, Iden node_list = std::move(result_node_list); } -/** Initialize query FROM section. +/** Initialize query join tree node. + * + * 1. Resolve identifiers. + * 2. Register table, table function, query nodes in scope table expressions in resolve process. */ -void QueryAnalyzer::initializeQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope) +void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope) { - if (from_node->getNodeType() == QueryTreeNodeType::QUERY || - from_node->getNodeType() == QueryTreeNodeType::TABLE) - { - /// Already initialized - } - else if (auto * table_function = from_node->as()) - { - /// Already initialized - } - else if (auto * from_table_identifier = from_node->as()) - { - auto table_identifier_lookup = IdentifierLookup{from_table_identifier->getIdentifier(), IdentifierLookupContext::TABLE}; - auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope); - auto resolved_identifier = table_identifier_resolve_result.resolved_identifier; + std::deque join_tree_node_ptrs_to_process_queue; + join_tree_node_ptrs_to_process_queue.push_back(&join_tree_node); - if (!resolved_identifier) - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Unknown {} identifier {} in scope {}", - toStringLowercase(IdentifierLookupContext::TABLE), - from_table_identifier->getIdentifier().getFullName(), - scope.scope_node->formatASTForErrorMessage()); + while (!join_tree_node_ptrs_to_process_queue.empty()) + { + auto * current_join_tree_node_ptr = join_tree_node_ptrs_to_process_queue.front(); + join_tree_node_ptrs_to_process_queue.pop_front(); - bool resolved_as_cte = resolved_identifier->as() && resolved_identifier->as()->isCTE(); + auto & current_join_tree_node = *current_join_tree_node_ptr; + auto current_join_tree_node_type = current_join_tree_node->getNodeType(); - if (resolved_as_cte) + switch (current_join_tree_node_type) { - resolved_identifier = resolved_identifier->clone(); - resolved_identifier->as().setIsCTE(false); + case QueryTreeNodeType::IDENTIFIER: + { + auto & from_table_identifier = current_join_tree_node->as(); + auto table_identifier_lookup = IdentifierLookup{from_table_identifier.getIdentifier(), IdentifierLookupContext::TABLE}; + + IdentifierResolveSettings resolve_settings; + /// In join tree initialization ignore join tree as identifier lookup source + resolve_settings.allow_to_check_join_tree = false; + + auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope, resolve_settings); + auto resolved_identifier = table_identifier_resolve_result.resolved_identifier; + + if (!resolved_identifier) + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Unknown {} identifier {} in scope {}", + toStringLowercase(IdentifierLookupContext::TABLE), + from_table_identifier.getIdentifier().getFullName(), + scope.scope_node->formatASTForErrorMessage()); + + /** SELECT a.id, b.id FROM test_table_join_1 AS a INNER JOIN test_table_join_1 AS b. + * Because of identifier cache we receive same result for same identifier. + * Use table_expressions_in_resolve_process to detect if resolved identifier must be cloned. + */ + if (scope.table_expressions_in_resolve_process.contains(resolved_identifier.get())) + { + resolved_identifier = resolved_identifier->clone(); + resolved_identifier->removeAlias(); + } + + bool resolved_as_cte = resolved_identifier->as() && resolved_identifier->as()->isCTE(); + + if (resolved_as_cte) + { + resolved_identifier = resolved_identifier->clone(); + resolved_identifier->as().setIsCTE(false); + } + + auto current_join_tree_node_alias = current_join_tree_node->getAlias(); + resolved_identifier->setAlias(current_join_tree_node_alias); + current_join_tree_node = resolved_identifier; + + scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); + break; + } + case QueryTreeNodeType::QUERY: + { + scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); + break; + } + case QueryTreeNodeType::TABLE_FUNCTION: + { + scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); + break; + } + case QueryTreeNodeType::TABLE: + { + scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + auto & array_join = current_join_tree_node->as(); + join_tree_node_ptrs_to_process_queue.push_back(&array_join.getTableExpression()); + break; + } + case QueryTreeNodeType::JOIN: + { + auto & join = current_join_tree_node->as(); + join_tree_node_ptrs_to_process_queue.push_back(&join.getLeftTableExpression()); + join_tree_node_ptrs_to_process_queue.push_back(&join.getRightTableExpression()); + break; + } + default: + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Query FROM section expected table, table function, query, ARRAY JOIN or JOIN. Actual {} {}. In scope {}", + current_join_tree_node->getNodeTypeName(), + current_join_tree_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } } - - auto from_node_alias = from_node->getAlias(); - resolved_identifier->setAlias(from_node_alias); - from_node = resolved_identifier; - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Query FROM section expected table or query. Actual {} {}. In scope {}", - from_node->getNodeTypeName(), - from_node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - } - - auto from_node_alias = from_node->getAlias(); - if (!from_node_alias.empty()) - { - auto [_, inserted] = scope.alias_name_to_table_expression_node.insert(std::make_pair(from_node_alias, from_node)); - if (!inserted) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Duplicate aliases for tables in FROM section are not allowed", - from_node_alias); } } -/** Resolve query FROM section. - * - * Resolve steps: - * 1. If FROM node is query, resolve it. - * 2. If FROM node is table is it already resolved. - */ -void QueryAnalyzer::resolveQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope) +/// Initialize table expression columns for table expression node +void QueryAnalyzer::initializeTableExpressionColumns(QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) { - if (from_node->getNodeType() == QueryTreeNodeType::QUERY) - { - IdentifierResolveScope subquery_scope(from_node, &scope); - resolveQuery(from_node, subquery_scope); - return; - } - else if (auto * from_table_function = from_node->as()) - { - const auto & table_function_factory = TableFunctionFactory::instance(); - const auto & table_function_name = from_table_function->getTableFunctionName(); + auto * table_node = table_expression_node->as(); + auto * query_node = table_expression_node->as(); + auto * table_function_node = table_expression_node->as(); - TableFunctionPtr table_function_ptr = table_function_factory.tryGet(table_function_name, context); - if (!table_function_ptr) + if (!table_node && !table_function_node && !query_node) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Unexpected table expression. Expected table, table function or query node. Actual {}. In scope {}", + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + auto storage_columns_it = table_expression_node_to_columns.find(table_expression_node.get()); + if (storage_columns_it != table_expression_node_to_columns.end()) + return; + + TableExpressionColumns storage_columns; + + if (table_node || table_function_node) + { + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + + auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); + const auto & columns_description = storage_snapshot->metadata->getColumns(); + + std::vector> alias_columns_to_resolve; + std::unordered_map column_name_to_column_node; + column_name_to_column_node.reserve(column_names_and_types.size()); + + for (const auto & column_name_and_type : column_names_and_types) { - auto hints = TableFunctionFactory::instance().getHints(table_function_name); - if (!hints.empty()) - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}. Maybe you meant: {}", table_function_name, toString(hints)); + const auto & column_default = columns_description.getDefault(column_name_and_type.name); + + if (column_default && column_default->kind == ColumnDefaultKind::Alias) + { + auto column_node = std::make_shared(column_name_and_type, buildQueryTree(column_default->expression, context), table_expression_node); + column_name_to_column_node.emplace(column_name_and_type.name, column_node); + alias_columns_to_resolve.emplace_back(column_name_and_type.name, column_node); + } else - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", table_function_name); - } - - if (context->getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint()) - { - const auto & insertion_table = context->getInsertionTable(); - if (!insertion_table.empty()) { - const auto & structure_hint = DatabaseCatalog::instance().getTable(insertion_table, context)->getInMemoryMetadataPtr()->columns; - table_function_ptr->setStructureHint(structure_hint); + auto column_node = std::make_shared(column_name_and_type, table_expression_node); + column_name_to_column_node.emplace(column_name_and_type.name, column_node); } } - /// TODO: Special functions that can take query - /// TODO: Support qualified matchers for table function - - for (auto & argument_node : from_table_function->getArguments().getNodes()) + for (auto & [alias_column_to_resolve_name, alias_column_to_resolve] : alias_columns_to_resolve) { - if (argument_node->getNodeType() == QueryTreeNodeType::MATCHER) + /** Alias column could be potentially resolved during resolve of other ALIAS column. + * Example: CREATE TABLE test_table (id UInt64, alias_value_1 ALIAS id + alias_value_2, alias_value_2 ALIAS id + 5) ENGINE=TinyLog; + * + * During resolve of alias_value_1, alias_value_2 column will be resolved. + */ + alias_column_to_resolve = column_name_to_column_node[alias_column_to_resolve_name]; + + IdentifierResolveScope alias_column_resolve_scope(alias_column_to_resolve, nullptr /*parent_scope*/); + alias_column_resolve_scope.column_name_to_column_node = std::move(column_name_to_column_node); + + /// Initialize aliases in alias column scope + QueryExpressionsAliasVisitor::Data data{alias_column_resolve_scope}; + QueryExpressionsAliasVisitor visitor(data); + + visitor.visit(alias_column_to_resolve->getExpression()); + + resolveExpressionNode(alias_column_resolve_scope.scope_node, + alias_column_resolve_scope, + false /*allow_lambda_expression*/, + false /*allow_table_expression*/); + + column_name_to_column_node = std::move(alias_column_resolve_scope.column_name_to_column_node); + column_name_to_column_node[alias_column_to_resolve_name] = alias_column_to_resolve; + } + + storage_columns.column_name_to_column_node = std::move(column_name_to_column_node); + } + else if (query_node) + { + auto column_names_and_types = query_node->computeProjectionColumns(); + storage_columns.column_name_to_column_node.reserve(column_names_and_types.size()); + + for (const auto & column_name_and_type : column_names_and_types) + { + auto column_node = std::make_shared(column_name_and_type, table_expression_node); + storage_columns.column_name_to_column_node.emplace(column_name_and_type.name, column_node); + } + } + + storage_columns.column_identifier_first_parts.reserve(storage_columns.column_name_to_column_node.size()); + + for (auto & [column_name, _] : storage_columns.column_name_to_column_node) + { + Identifier column_name_identifier(column_name); + storage_columns.column_identifier_first_parts.insert(column_name_identifier.at(0)); + } + + table_expression_node_to_columns.emplace(table_expression_node.get(), std::move(storage_columns)); +} + +/** Resolve query join tree. + * + * Query join tree must be initialized before calling this function. + */ +void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope, QueryExpressionsAliasVisitor & expressions_visitor) +{ + auto add_table_expression_alias_into_scope = [&](const QueryTreeNodePtr & table_expression_node) + { + const auto & alias_name = table_expression_node->getAlias(); + if (alias_name.empty()) + return; + + auto [it, inserted] = scope.alias_name_to_table_expression_node.emplace(alias_name, table_expression_node); + if (!inserted) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Duplicate aliases {} for table expressions in FROM section are not allowed. Try to register {}. Already registered {}.", + alias_name, + table_expression_node->formatASTForErrorMessage(), + it->second->formatASTForErrorMessage()); + }; + + auto from_node_type = join_tree_node->getNodeType(); + + switch (from_node_type) + { + case QueryTreeNodeType::QUERY: + { + IdentifierResolveScope subquery_scope(join_tree_node, &scope); + resolveQuery(join_tree_node, subquery_scope); + break; + } + case QueryTreeNodeType::TABLE_FUNCTION: + { + auto & table_function_node = join_tree_node->as(); + expressions_visitor.visit(table_function_node.getArgumentsNode()); + + const auto & table_function_factory = TableFunctionFactory::instance(); + const auto & table_function_name = table_function_node.getTableFunctionName(); + + TableFunctionPtr table_function_ptr = table_function_factory.tryGet(table_function_name, context); + if (!table_function_ptr) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Matcher as table function argument is not supported {}. In scope {}", - from_node->formatASTForErrorMessage(), + auto hints = TableFunctionFactory::instance().getHints(table_function_name); + if (!hints.empty()) + throw Exception( + ErrorCodes::UNKNOWN_FUNCTION, + "Unknown table function {}. Maybe you meant: {}", + table_function_name, + DB::toString(hints)); + else + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", table_function_name); + } + + if (context->getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint()) + { + const auto & insertion_table = context->getInsertionTable(); + if (!insertion_table.empty()) + { + const auto & structure_hint + = DatabaseCatalog::instance().getTable(insertion_table, context)->getInMemoryMetadataPtr()->columns; + table_function_ptr->setStructureHint(structure_hint); + } + } + + /// TODO: Special functions that can take query + /// TODO: Support qualified matchers for table function + + for (auto & argument_node : table_function_node.getArguments().getNodes()) + { + if (argument_node->getNodeType() == QueryTreeNodeType::MATCHER) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Matcher as table function argument is not supported {}. In scope {}", + join_tree_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + + resolveExpressionNode(argument_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + } + + auto table_function_ast = table_function_node.toAST(); + table_function_ptr->parseArguments(table_function_ast, context); + + auto table_function_storage = table_function_ptr->execute(table_function_ast, context, table_function_ptr->getName()); + table_function_node.resolve(std::move(table_function_ptr), std::move(table_function_storage), context); + break; + } + case QueryTreeNodeType::TABLE: + { + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + auto & array_join_node = join_tree_node->as(); + resolveQueryJoinTreeNode(array_join_node.getTableExpression(), scope, expressions_visitor); + + for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) + { + auto array_join_expression_alias = array_join_expression->getAlias(); + + if (!array_join_expression_alias.empty() && scope.alias_name_to_expression_node.contains(array_join_expression_alias)) + throw Exception( + ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, + "ARRAY JOIN expression {} with duplicate alias {}. In scope {}", + array_join_expression->formatASTForErrorMessage(), + array_join_expression_alias, + scope.scope_node->formatASTForErrorMessage()); + } + + /// Add array join expressions into scope + + expressions_visitor.visit(array_join_node.getJoinExpressionsNode()); + resolveExpressionNode(array_join_node.getJoinExpressionsNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + /// Wrap array join expressions into column nodes, where array join expression is inner expression. + + for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) + { + auto array_join_expression_alias = array_join_expression->getAlias(); + + auto result_name = "__array_join_expression_" + std::to_string(array_join_expressions_counter); + ++array_join_expressions_counter; + + auto result_type = array_join_expression->getResultType(); + + if (!isArray(result_type)) + throw Exception( + ErrorCodes::TYPE_MISMATCH, + "ARRAY JOIN requires expression with Array type. Actual {}. In scope {}", + result_type->getName(), + scope.scope_node->formatASTForErrorMessage()); + + array_join_expression->removeAlias(); + + result_type = assert_cast(*result_type).getNestedType(); + + auto array_join_column = std::make_shared( + NameAndTypePair{result_name, result_type}, array_join_expression, join_tree_node); + array_join_expression = std::move(array_join_column); + array_join_expression->setAlias(array_join_expression_alias); + + auto it = scope.alias_name_to_expression_node.find(array_join_expression_alias); + if (it != scope.alias_name_to_expression_node.end()) + it->second = array_join_expression; + } + + break; + } + case QueryTreeNodeType::JOIN: + { + auto & join_node = join_tree_node->as(); + + resolveQueryJoinTreeNode(join_node.getLeftTableExpression(), scope, expressions_visitor); + resolveQueryJoinTreeNode(join_node.getRightTableExpression(), scope, expressions_visitor); + + if (join_node.isUsingJoinExpression()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "JOIN {} with USING is unsupported. In scope {}", + join_node.formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); + + if (join_node.getJoinExpression()) + { + expressions_visitor.visit(join_node.getJoinExpression()); + resolveExpressionNode(join_node.getJoinExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } - resolveExpressionNode(argument_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + break; } + case QueryTreeNodeType::IDENTIFIER: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Identifiers in FROM section must be already resolved. In scope {}", + join_tree_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + default: + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Query FROM section expected table, table function, query, ARRAY JOIN or JOIN. Actual {}. In scope {}", + join_tree_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + } - auto table_function_ast = from_table_function->toAST(); - table_function_ptr->parseArguments(table_function_ast, context); + if (isTableExpression(join_tree_node.get())) + initializeTableExpressionColumns(join_tree_node, scope); - auto table_function_storage = table_function_ptr->execute(table_function_ast, context, table_function_ptr->getName()); - from_table_function->resolve(std::move(table_function_ptr), std::move(table_function_storage), context); - } - else if (auto * from_table = from_node->as()) - { - return; - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Query FROM section expected table or query. Actual {}", from_node->formatASTForErrorMessage()); - } + add_table_expression_alias_into_scope(join_tree_node); + scope.table_expressions_in_resolve_process.erase(join_tree_node.get()); } /** Resolve query. @@ -2823,13 +3188,13 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /// Initialize aliases in query node scope - ScopeAliasVisitorMatcher::Data data{scope}; - ScopeAliasVisitorMatcher::Visitor visitor(data); + QueryExpressionsAliasVisitor::Data data{scope}; + QueryExpressionsAliasVisitor visitor(data); - if (query_node_typed.getWithNode()) + if (!query_node_typed.getWith().getNodes().empty()) visitor.visit(query_node_typed.getWithNode()); - if (query_node_typed.getProjectionNode()) + if (!query_node_typed.getProjection().getNodes().empty()) visitor.visit(query_node_typed.getProjectionNode()); if (query_node_typed.getPrewhere()) @@ -2840,35 +3205,33 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /// Register CTE subqueries - if (query_node_typed.getWithNode()) + for (auto & node : query_node_typed.getWith().getNodes()) { - for (auto & node : query_node_typed.getWith().getNodes()) - { - auto * subquery_node = node->as(); - if (!subquery_node || !subquery_node->isCTE()) - continue; + auto * subquery_node = node->as(); + if (!subquery_node || !subquery_node->isCTE()) + continue; - const auto & cte_name = subquery_node->getCTEName(); - auto [_, inserted] = data.scope.cte_name_to_query_node.emplace(cte_name, node); - if (!inserted) - throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, - "CTE with name {} already exists. In scope {}", - cte_name, - data.scope.scope_node->formatASTForErrorMessage()); - } + const auto & cte_name = subquery_node->getCTEName(); + auto [_, inserted] = data.scope.cte_name_to_query_node.emplace(cte_name, node); + if (!inserted) + throw Exception( + ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, + "CTE with name {} already exists. In scope {}", + cte_name, + data.scope.scope_node->formatASTForErrorMessage()); } if (query_node_typed.getFrom()) { - if (auto * table_function = query_node_typed.getFrom()->as()) - visitor.visit(table_function->getArgumentsNode()); + TableExpressionsAliasVisitor::Data table_expressions_visitor_data{scope}; + TableExpressionsAliasVisitor table_expressions_visitor(table_expressions_visitor_data); - auto [it, _] = query_node_from_section_in_resolve_process.emplace(query_node.get()); + table_expressions_visitor.visit(query_node_typed.getFrom()); - initializeQueryFrom(query_node_typed.getFrom(), scope); - resolveQueryFrom(query_node_typed.getFrom(), scope); + initializeQueryJoinTreeNode(query_node_typed.getFrom(), scope); + scope.alias_name_to_table_expression_node.clear(); - query_node_from_section_in_resolve_process.erase(it); + resolveQueryJoinTreeNode(query_node_typed.getFrom(), scope, visitor); } /** Clear cache of identifier lookups. @@ -2895,7 +3258,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.getWhere()) resolveExpressionNode(query_node_typed.getWhere(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - /** WITH section can be safely removed, because WITH section only can provide aliases to expressions + /** WITH section can be safely removed, because WITH section only can provide aliases to query expressions * and CTE for other sections to use. * * Example: WITH 1 AS constant, (x -> x + 1) AS lambda, a AS (SELECT * FROM test_table); @@ -2913,7 +3276,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier { auto node = node_with_duplicated_alias; auto node_alias = node->getAlias(); - resolveExpressionNode(node, scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); + resolveExpressionNode(node, scope, true /*allow_lambda_expression*/, true /*allow_table_expression*/); bool has_node_in_alias_table = false; @@ -2954,6 +3317,8 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier } } +} + void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { if (query_tree_node->getNodeType() != QueryTreeNodeType::QUERY) diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 83a2d6d2698..36972fed934 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -1,16 +1,22 @@ #include +#include + +#include + +#include +#include +#include + #include #include #include #include #include +#include +#include -#include - -#include -#include -#include +#include namespace DB { @@ -22,6 +28,29 @@ QueryNode::QueryNode() children[projection_child_index] = std::make_shared(); } +NamesAndTypesList QueryNode::computeProjectionColumns() const +{ + NamesAndTypes query_columns; + + const auto & projection_nodes = getProjection(); + query_columns.reserve(projection_nodes.getNodes().size()); + + for (const auto & projection_node : projection_nodes.getNodes()) + { + auto column_type = projection_node->getResultType(); + std::string column_name; + + if (projection_node->hasAlias()) + column_name = projection_node->getAlias(); + else + column_name = projection_node->getName(); + + query_columns.emplace_back(column_name, column_type); + } + + return {query_columns.begin(), query_columns.end()}; +} + String QueryNode::getName() const { WriteBufferFromOwnString buffer; @@ -121,21 +150,8 @@ ASTPtr QueryNode::toASTImpl() const select_query->setExpression(ASTSelectQuery::Expression::SELECT, children[projection_child_index]->toAST()); - auto table_expression_ast = std::make_shared(); - table_expression_ast->children.push_back(children[from_child_index]->toAST()); - - if (children[from_child_index]->getNodeType() == QueryTreeNodeType::TABLE) - table_expression_ast->database_and_table_name = table_expression_ast->children.back(); - else if (children[from_child_index]->getNodeType() == QueryTreeNodeType::QUERY) - table_expression_ast->subquery = table_expression_ast->children.back(); - - auto tables_in_select_query_element_ast = std::make_shared(); - tables_in_select_query_element_ast->children.push_back(std::move(table_expression_ast)); - tables_in_select_query_element_ast->table_expression = tables_in_select_query_element_ast->children.back(); - - auto tables_in_select_query_ast = std::make_shared(); - tables_in_select_query_ast->children.push_back(std::move(tables_in_select_query_element_ast)); - + ASTPtr tables_in_select_query_ast = std::make_shared(); + addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[from_child_index]); select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select_query_ast)); if (getPrewhere()) diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 78941cb2cae..a3b9ebaedd6 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -107,6 +107,11 @@ public: return children[prewhere_child_index]; } + bool hasWhere() const + { + return children[where_child_index] != nullptr; + } + const QueryTreeNodePtr & getWhere() const { return children[where_child_index]; @@ -117,6 +122,9 @@ public: return children[where_child_index]; } + /// Compute query node columns using projection section + NamesAndTypesList computeProjectionColumns() const; + QueryTreeNodeType getNodeType() const override { return QueryTreeNodeType::QUERY; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 2385c8e74d2..cf6f541cbfb 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -31,6 +31,8 @@ #include #include #include +#include +#include #include #include @@ -360,8 +362,8 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q } auto & tables = tables_in_select_query->as(); - if (tables.children.size() > 1) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Multiple tables are not supported"); + + QueryTreeNodes table_expressions; for (const auto & table_element_untyped : tables.children) { @@ -380,7 +382,7 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q node->setAlias(table_identifier_typed.tryGetAlias()); node->setOriginalAST(table_element.table_expression); - return node; + table_expressions.push_back(std::move(node)); } else if (table_expression.subquery) { @@ -392,7 +394,7 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q node->setAlias(subquery_expression.tryGetAlias()); node->setOriginalAST(select_with_union_query); - return node; + table_expressions.push_back(std::move(node)); } else if (table_expression.table_function) { @@ -410,59 +412,68 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q node->setAlias(table_function_expression.tryGetAlias()); node->setOriginalAST(table_expression.table_function); - return node; + table_expressions.push_back(std::move(node)); + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Unsupported table expression node {}", table_element.table_expression->formatForErrorMessage()); } - - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Unsupported table expression node {}", table_element.table_expression->formatForErrorMessage()); } - // if (table_element.table_join) - // { - // const auto & table_join = table_element.table_join->as(); + if (table_element.table_join) + { + const auto & table_join = table_element.table_join->as(); - // auto right_table_expression = std::move(table_expressions.back()); - // table_expressions.pop_back(); + auto right_table_expression = std::move(table_expressions.back()); + table_expressions.pop_back(); - // auto left_table_expression = std::move(table_expressions.back()); - // table_expressions.pop_back(); + auto left_table_expression = std::move(table_expressions.back()); + table_expressions.pop_back(); - // auto join_expression = JoinExpression::create(); - // join_expression->getLeftTableExpression() = left_table_expression; - // join_expression->getRightTableExpression() = right_table_expression; + QueryTreeNodePtr join_expression; - // if (table_join->using_expression_list) - // join_expression->getUsingExpressions() = getExpressionElements(table_join->using_expression_list, scope); + if (table_join.using_expression_list) + join_expression = getExpressionList(table_join.using_expression_list); + else if (table_join.on_expression) + join_expression = getExpression(table_join.on_expression); - // if (table_join->on_expression) - // { - // join_expression->getOnExpression() = getExpressionElement(table_join->on_expression, scope); - // } + auto join_node = std::make_shared(std::move(left_table_expression), + std::move(right_table_expression), + std::move(join_expression), + table_join.locality, + table_join.strictness, + table_join.kind); - // table_expressions.emplace_back(std::move(join_expression)); - // } + /** Original AST is not set because it will contain only join part and does + * not include left table expression. + */ + table_expressions.emplace_back(std::move(join_node)); + } - // if (table_element.array_join) - // { - // auto array_join_array_expression = table_element.array_join->children[0]->children[0]; - // auto expression_element = getExpressionElement(array_join_array_expression, scope); - // expression_element->setAlias(array_join_array_expression->tryGetAlias()); + if (table_element.array_join) + { + auto & array_join_expression = table_element.array_join->as(); + bool is_left_array_join = array_join_expression.kind == ASTArrayJoin::Kind::Left; - // auto last_table_expression = std::move(table_expressions.back()); - // table_expressions.pop_back(); + auto last_table_expression = std::move(table_expressions.back()); + table_expressions.pop_back(); - // auto array_join_expression = ArrayJoinExpression::create(); - // array_join_expression->getLeftTableExpression() = std::move(last_table_expression); - // array_join_expression->getArrayExpression() = std::move(expression_element); + auto array_join_expressions_list = getExpressionList(array_join_expression.expression_list); - // table_expressions.push_back(array_join_expression); - // } + auto array_join_node = std::make_shared(std::move(last_table_expression), std::move(array_join_expressions_list), is_left_array_join); + array_join_node->setOriginalAST(table_element.array_join); + + table_expressions.push_back(std::move(array_join_node)); + } } - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "From section is unsupported"); - // if (table_expressions.empty()) - // throw Exception(ErrorCodes::LOGICAL_ERROR, "QueryAnalyzer from cannot be empty"); + if (table_expressions.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query FROM section cannot be empty"); - // return table_expressions.back(); + if (table_expressions.size() > 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query FROM section cannot have more than 1 root table expression"); + + return table_expressions.back(); } diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index cc5de07b916..768e6f0e6a3 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -13,11 +13,11 @@ namespace DB { -TableNode::TableNode(StoragePtr storage_, ContextPtr context) +TableNode::TableNode(StoragePtr storage_, TableLockHolder storage_lock_, StorageSnapshotPtr storage_snapshot_) : storage(std::move(storage_)) , storage_id(storage->getStorageID()) - , table_lock(storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)) - , storage_snapshot(storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context)) + , storage_lock(std::move(storage_lock_)) + , storage_snapshot(std::move(storage_snapshot_)) { } @@ -62,7 +62,7 @@ QueryTreeNodePtr TableNode::cloneImpl() const result_table_node->storage = storage; result_table_node->storage_id = storage_id; - result_table_node->table_lock = table_lock; + result_table_node->storage_lock = storage_lock; result_table_node->storage_snapshot = storage_snapshot; return result_table_node; diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index 6756381ed34..0a528b4db67 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -26,8 +26,8 @@ using TableNodePtr = std::shared_ptr; class TableNode : public IQueryTreeNode { public: - /// Construct table node with storage and context - explicit TableNode(StoragePtr storage_, ContextPtr context); + /// Construct table node with storage, storage lock, storage snapshot + explicit TableNode(StoragePtr storage_, TableLockHolder storage_lock_, StorageSnapshotPtr storage_snapshot_); /// Get storage const StoragePtr & getStorage() const @@ -47,18 +47,18 @@ public: return storage_snapshot; } - /// Get table lock - const TableLockHolder & getTableLock() const + /// Get storage lock + const TableLockHolder & getStorageLock() const { - return table_lock; + return storage_lock; } /** Move table lock out of table node. * After using this method table node state becomes invalid. */ - TableLockHolder && moveTableLock() + TableLockHolder && moveStorageLock() { - return std::move(table_lock); + return std::move(storage_lock); } QueryTreeNodeType getNodeType() const override @@ -84,7 +84,7 @@ private: StoragePtr storage; StorageID storage_id; - TableLockHolder table_lock; + TableLockHolder storage_lock; StorageSnapshotPtr storage_snapshot; }; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 1aade42351f..75b3f37aaad 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -1,5 +1,12 @@ #include +#include +#include +#include +#include + +#include + namespace DB { @@ -36,4 +43,91 @@ bool isNameOfInFunction(const std::string & function_name) return is_special_function_in; } +bool isTableExpression(const IQueryTreeNode * node) +{ + auto node_type = node->getNodeType(); + return node_type == QueryTreeNodeType::TABLE || node_type == QueryTreeNodeType::TABLE_FUNCTION || node_type == QueryTreeNodeType::QUERY; +} + +static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expression_node) +{ + ASTPtr table_expression_node_ast; + auto node_type = table_expression_node->getNodeType(); + + if (node_type == QueryTreeNodeType::IDENTIFIER) + { + const auto & identifier_node = table_expression_node->as(); + const auto & identifier = identifier_node.getIdentifier(); + + if (identifier.getPartsSize() == 1) + table_expression_node_ast = std::make_shared(identifier[0]); + else if (identifier.getPartsSize() == 2) + table_expression_node_ast = std::make_shared(identifier[0], identifier[1]); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Identifier for table expression must contain 1 or 2 parts. Actual {}", + identifier.getFullName()); + } + else + { + table_expression_node_ast = table_expression_node->toAST(); + } + + auto result_table_expression = std::make_shared(); + result_table_expression->children.push_back(table_expression_node_ast); + + if (node_type == QueryTreeNodeType::QUERY) + result_table_expression->subquery = result_table_expression->children.back(); + else if (node_type == QueryTreeNodeType::TABLE || node_type == QueryTreeNodeType::IDENTIFIER) + result_table_expression->database_and_table_name = result_table_expression->children.back(); + else if (node_type == QueryTreeNodeType::TABLE_FUNCTION) + result_table_expression->table_function = result_table_expression->children.back(); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected identiifer, table, query, or table function. Actual {}", table_expression_node->formatASTForErrorMessage()); + + return result_table_expression; +} + +void addTableExpressionIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression) +{ + auto table_expression_node_type = table_expression->getNodeType(); + + switch (table_expression_node_type) + { + case QueryTreeNodeType::IDENTIFIER: + [[fallthrough]]; + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + auto table_expression_ast = convertIntoTableExpressionAST(table_expression); + + auto tables_in_select_query_element_ast = std::make_shared(); + tables_in_select_query_element_ast->children.push_back(std::move(table_expression_ast)); + tables_in_select_query_element_ast->table_expression = tables_in_select_query_element_ast->children.back(); + + tables_in_select_query_ast->children.push_back(std::move(tables_in_select_query_element_ast)); + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + [[fallthrough]]; + case QueryTreeNodeType::JOIN: + { + auto table_expression_tables_in_select_query_ast = table_expression->toAST(); + tables_in_select_query_ast->children.reserve(table_expression_tables_in_select_query_ast->children.size()); + for (auto && left_table_element_ast : table_expression_tables_in_select_query_ast->children) + tables_in_select_query_ast->children.push_back(std::move(left_table_element_ast)); + break; + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected node type for table expression. Expected table, query, table function, join or array join. Actual {}", + table_expression->getNodeTypeName()); + } + } +} + } diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index d93d81d654e..015233a05dc 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -11,4 +11,12 @@ bool isNodePartOfTree(const IQueryTreeNode * node, const IQueryTreeNode * root); /// Returns true if function name is name of IN function or its variations, false otherwise bool isNameOfInFunction(const std::string & function_name); +/// Returns true if node hase type table, table function, or query, false otherwise +bool isTableExpression(const IQueryTreeNode * node); + +/** Add table expression in tables in select query children. + * If table expression node is not of identifier node, table node, query node, table function node, join node or array join node type throws logical error exception. + */ +void addTableExpressionIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression); + } diff --git a/src/Analyzer/examples/query_analyzer.cpp b/src/Analyzer/examples/query_analyzer.cpp index 0e9073306a0..5a20b46b346 100644 --- a/src/Analyzer/examples/query_analyzer.cpp +++ b/src/Analyzer/examples/query_analyzer.cpp @@ -1,7 +1,5 @@ #include -using namespace DB; - int main(int argc, char ** argv) { (void)(argc); diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index b27df0f1c35..9b38072b5af 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1073,8 +1073,8 @@ void ExpressionActionsChain::JoinStep::finalize(const NameSet & required_output_ } /// Result will also contain joined columns. - for (const auto & column_name : analyzed_join->columnsAddedByJoin()) - required_names.emplace(column_name); + for (const auto & column : analyzed_join->columnsAddedByJoin()) + required_names.emplace(column.name); for (const auto & column : result_columns) { diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index baba2214333..d3d13d80f57 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -417,8 +418,17 @@ QueryPipeline InterpreterExplainQuery::executeImpl() auto settings = checkAndGetSettings(ast.getSettings()); QueryPlan plan; - InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options); - interpreter.buildQueryPlan(plan); + if (getContext()->getSettingsRef().use_analyzer) + { + InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), options, getContext()); + interpreter.initializeQueryPlanIfNeeded(); + plan = std::move(interpreter).extractQueryPlan(); + } + else + { + InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options); + interpreter.buildQueryPlan(plan); + } if (settings.optimize) plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext())); @@ -452,8 +462,18 @@ QueryPipeline InterpreterExplainQuery::executeImpl() auto settings = checkAndGetSettings(ast.getSettings()); QueryPlan plan; - InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options); - interpreter.buildQueryPlan(plan); + if (getContext()->getSettingsRef().use_analyzer) + { + InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), options, getContext()); + interpreter.initializeQueryPlanIfNeeded(); + plan = std::move(interpreter).extractQueryPlan(); + } + else + { + InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options); + interpreter.buildQueryPlan(plan); + } + auto pipeline = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(getContext()), BuildQueryPipelineSettings::fromContext(getContext())); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index b407a77c50c..c2b2d95a94d 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -21,11 +22,14 @@ #include #include #include +#include +#include #include #include #include #include +#include #include #include @@ -34,9 +38,14 @@ #include #include #include +#include +#include #include #include +#include +#include +#include namespace DB @@ -48,33 +57,37 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; + extern const int INVALID_JOIN_ON_EXPRESSION; } -QueryPipeline buildDummyPipeline() +/** ClickHouse query planner. + * + * TODO: JOIN support columns cast. JOIN support ASOF. JOIN support strictness. + * TODO: Support RBAC. Support RBAC for ALIAS columns. + * TODO: Support distributed query processing + * TODO: Support PREWHERE + * TODO: Support GROUP BY, HAVING + * TODO: Support ORDER BY, LIMIT + * TODO: Support WINDOW FUNCTIONS + * TODO: Support DISTINCT + * TODO: Support ArrayJoin + * TODO: Support building sets for IN functions + * TODO: Support trivial count optimization + * TODO: Support totals, extremes + * TODO: Support projections + */ + +namespace { - ColumnsWithTypeAndName columns; - auto string_data_type = std::make_shared(); - auto string_column = string_data_type->createColumn(); - string_column->insert("TestValue"); - columns.emplace_back(ColumnWithTypeAndName{std::move(string_column), string_data_type, "test_column"}); - - Block block(columns); - auto source = std::make_shared(block); - auto shell_input_pipe = Pipe(std::move(source)); - - QueryPipeline pipeline(std::move(shell_input_pipe)); - return pipeline; -} - -String dumpQueryPlan(QueryPlan & query_plan) +[[maybe_unused]] String dumpQueryPlan(QueryPlan & query_plan) { WriteBufferFromOwnString query_plan_buffer; - query_plan.explainPlan(query_plan_buffer, QueryPlan::ExplainPlanOptions{}); + query_plan.explainPlan(query_plan_buffer, QueryPlan::ExplainPlanOptions{true, true, true, true}); return query_plan_buffer.str(); } -String dumpQueryPipeline(QueryPlan & query_plan) +[[maybe_unused]] String dumpQueryPipeline(QueryPlan & query_plan) { QueryPlan::ExplainPipelineOptions explain_pipeline; WriteBufferFromOwnString query_pipeline_buffer; @@ -82,6 +95,434 @@ String dumpQueryPipeline(QueryPlan & query_plan) return query_pipeline_buffer.str(); } +struct TableExpressionColumns +{ + NamesAndTypesList all_columns; + NameSet all_columns_names_set; + + NamesAndTypesList source_input_columns; + NameSet source_columns_set; + std::unordered_map column_name_to_column_identifier; +}; + +using TableExpressionNodeToColumns = std::unordered_map; +using TableExpressionColumnNodeToColumnIdentifier = std::unordered_map; +using ActionsNodeNameToCount = std::unordered_map; + +class ActionsChainNode; +using ActionsChainNodePtr = std::unique_ptr; +using ActionsChainNodes = std::vector; + +class ActionsChainNode +{ +public: + explicit ActionsChainNode(ActionsDAGPtr actions_, bool available_output_columns_only_aliases_ = false) + : actions(std::move(actions_)) + , available_output_columns_only_aliases(available_output_columns_only_aliases_) + { + initialize(); + } + + [[maybe_unused]] ActionsDAGPtr & getActions() + { + return actions; + } + + [[maybe_unused]] const ActionsDAGPtr & getActions() const + { + return actions; + } + + const ColumnsWithTypeAndName & getAvailableOutputColumns() const + { + return available_output_columns; + } + + const NameSet & getInputColumnNames() const + { + return input_columns_names; + } + + const NameSet & getChildRequiredOutputColumnsNames() const + { + return child_required_output_columns_names; + } + + void finalizeInputAndOutputColumns(NameSet & child_input_columns) + { + child_required_output_columns_names.clear(); + std::vector required_output_nodes; + + for (const auto & node : actions->getNodes()) + { + auto it = child_input_columns.find(node.result_name); + + if (it == child_input_columns.end()) + continue; + + child_required_output_columns_names.insert(node.result_name); + required_output_nodes.push_back(&node); + child_input_columns.erase(it); + } + + for (auto & required_output_node : required_output_nodes) + actions->addOrReplaceInOutputs(*required_output_node); + + actions->removeUnusedActions(); + initialize(); + } + + void dump(WriteBuffer & buffer) const + { + buffer << "DAG" << '\n'; + buffer << actions->dumpDAG(); + if (!child_required_output_columns_names.empty()) + { + buffer << "Child required output columns " << boost::join(child_required_output_columns_names, ", "); + buffer << '\n'; + } + } + + [[maybe_unused]] String dump() const + { + WriteBufferFromOwnString buffer; + dump(buffer); + + return buffer.str(); + } + + // NamesAndTypes getAvailableOutputNamesAndTypes() const + // { + // NamesAndTypes result; + // result.reserve(available_output_columns.size()); + + // for (const auto & available_output_column : available_output_columns) + // result.emplace_back(available_output_column.name, available_output_column.type); + + // return result; + // } + + // [[maybe_unused]] Names getAvailableOutputNames() const + // { + // Names result; + // result.reserve(available_output_columns.size()); + + // for (const auto & available_output_column : available_output_columns) + // result.emplace_back(available_output_column.name); + + // return result; + // } + + [[maybe_unused]] void addParentIndex(size_t parent_node_index) + { + parent_nodes_indices.push_back(parent_node_index); + } + + void addParentIndices(const std::vector & parent_nodes_indices_value) + { + parent_nodes_indices.insert(parent_nodes_indices.end(), parent_nodes_indices_value.begin(), parent_nodes_indices_value.end()); + } + + const std::vector & getParentNodesIndices() const + { + return parent_nodes_indices; + } + +private: + void initialize() + { + auto required_columns_names = actions->getRequiredColumnsNames(); + input_columns_names = NameSet(required_columns_names.begin(), required_columns_names.end()); + + available_output_columns.clear(); + + for (const auto & node : actions->getNodes()) + { + if (available_output_columns_only_aliases) + { + if (node.type == ActionsDAG::ActionType::ALIAS) + available_output_columns.emplace_back(node.column, node.result_type, node.result_name); + + continue; + } + + if (node.type == ActionsDAG::ActionType::INPUT || + node.type == ActionsDAG::ActionType::FUNCTION || + node.type == ActionsDAG::ActionType::ARRAY_JOIN) + available_output_columns.emplace_back(node.column, node.result_type, node.result_name); + } + } + + ActionsDAGPtr actions; + + bool available_output_columns_only_aliases; + + NameSet input_columns_names; + + NameSet child_required_output_columns_names; + + ColumnsWithTypeAndName available_output_columns; + + std::vector parent_nodes_indices; + +}; + +class ActionsChain +{ +public: + void addNode(ActionsChainNodePtr node) + { + nodes.emplace_back(std::move(node)); + } + + [[maybe_unused]] const ActionsChainNodes & getNodes() const + { + return nodes; + } + + ColumnsWithTypeAndName getAvailableOutputColumns(const std::vector & nodes_indices) + { + ColumnsWithTypeAndName result; + + for (const auto & node_index : nodes_indices) + { + assert(node_index < nodes.size()); + const auto & node_available_output_columns = nodes[node_index]->getAvailableOutputColumns(); + result.insert(result.end(), node_available_output_columns.begin(), node_available_output_columns.end()); + } + + return result; + } + + // ColumnsWithTypeAndName getOutputColumns(const std::vector & nodes_indices) + // { + // ColumnsWithTypeAndName result; + + // for (const auto & node_index : nodes_indices) + // { + // assert(node_index < nodes.size()); + // const auto & node_output_columns = nodes[node_index]->getActions()->getResultColumns(); + // result.insert(result.end(), node_output_columns.begin(), node_output_columns.end()); + // } + + // return result; + // } + + [[maybe_unused]] NameSet getInputColumnNames(const std::vector & nodes_indices) + { + NameSet result; + + for (const auto & node_index : nodes_indices) + { + assert(node_index < nodes.size()); + const auto & node_input_column_names = nodes[node_index]->getInputColumnNames(); + result.insert(node_input_column_names.begin(), node_input_column_names.end()); + } + + return result; + } + + [[maybe_unused]] size_t size() const + { + return nodes.size(); + } + + [[maybe_unused]] const ActionsChainNodePtr & at(size_t index) const + { + if (index >= nodes.size()) + throw std::out_of_range("actions chain access is out of range"); + + return nodes[index]; + } + + [[maybe_unused]] ActionsChainNodePtr & at(size_t index) + { + if (index >= nodes.size()) + throw std::out_of_range("actions chain access is out of range"); + + return nodes[index]; + } + + [[maybe_unused]] ActionsChainNodePtr & operator[](size_t index) + { + return nodes[index]; + } + + [[maybe_unused]] const ActionsChainNodePtr & operator[](size_t index) const + { + return nodes[index]; + } + + [[maybe_unused]] ActionsChainNode * getLastNode() + { + return nodes.back().get(); + } + + [[maybe_unused]] ActionsChainNode * getLastNodeOrThrow() + { + if (nodes.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ActionsChain is empty"); + + return nodes.back().get(); + } + + size_t getLastNodeIndex() + { + return nodes.size() - 1; + } + + [[maybe_unused]] size_t getLastNodeIndexOrThrow() + { + if (nodes.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ActionsChain is empty"); + + return nodes.size() - 1; + } + + void finalize() + { + if (nodes.empty()) + return; + + std::deque nodes_indices_to_process; + nodes_indices_to_process.push_front(nodes.size() - 1); + + /// For root node there are no columns required in child nodes + NameSet empty_child_input_columns; + nodes.back().get()->finalizeInputAndOutputColumns(empty_child_input_columns); + + while (!nodes_indices_to_process.empty()) + { + auto node_index_to_process = nodes_indices_to_process.front(); + nodes_indices_to_process.pop_front(); + + auto & node_to_process = nodes[node_index_to_process]; + + const auto & parent_nodes_indices = node_to_process->getParentNodesIndices(); + auto input_columns_names_copy = node_to_process->getInputColumnNames(); + + for (const auto & parent_node_index : parent_nodes_indices) + { + assert(parent_node_index < nodes.size()); + + auto & parent_node = nodes[parent_node_index]; + parent_node->finalizeInputAndOutputColumns(input_columns_names_copy); + nodes_indices_to_process.push_back(parent_node_index); + } + } + } + + void dump(WriteBuffer & buffer) const + { + size_t nodes_size = nodes.size(); + + for (size_t i = 0; i < nodes_size; ++i) + { + const auto & node = nodes[i]; + buffer << "Node " << i; + + const auto & parent_nodes_indices = node->getParentNodesIndices(); + if (!parent_nodes_indices.empty()) + { + buffer << " parent nodes indices "; + for (const auto & parent_node_index : parent_nodes_indices) + buffer << parent_node_index << ' '; + } + + buffer << '\n'; + node->dump(buffer); + + buffer << '\n'; + } + } + + [[maybe_unused]] String dump() const + { + WriteBufferFromOwnString buffer; + dump(buffer); + return buffer.str(); + } + +private: + ActionsChainNodes nodes; +}; + +class QueryPlanBuilder +{ +public: + using BuildRootStep = std::function; + using UniteStep = std::function)>; + using BuildStep = std::function; + + explicit QueryPlanBuilder(QueryPlan plan_root_) + { + auto plan_root_ptr = std::make_shared(std::move(plan_root_)); + build_root_step = [plan_root_ptr]() + { + return std::move(*plan_root_ptr); + }; + } + + [[maybe_unused]] explicit QueryPlanBuilder(std::vector plan_builders_, UniteStep unit_step_) + { + auto plan_builders_ptr = std::make_shared>(std::move(plan_builders_)); + build_root_step = [plan_builders_ptr, unite_step = std::move(unit_step_)]() + { + auto plan_builders = std::move(*plan_builders_ptr); + std::vector plans; + plans.reserve(plan_builders.size()); + + for (auto && plan_builder : plan_builders) + plans.push_back(std::move(plan_builder).buildPlan()); + + return unite_step(std::move(plans)); + }; + } + + QueryPlanBuilder(QueryPlanBuilder &&) noexcept = default; + [[maybe_unused]] QueryPlanBuilder & operator=(QueryPlanBuilder &&) noexcept = default; + + void addBuildStep(BuildStep step) + { + build_steps.push_back(std::move(step)); + } + + QueryPlan buildPlan() && + { + auto plan = build_root_step(); + + for (auto & build_step : build_steps) + build_step(plan); + + return plan; + } +private: + BuildRootStep build_root_step; + std::vector build_steps; +}; + +struct PlannerContext +{ + TableExpressionColumnNodeToColumnIdentifier table_expression_column_node_to_column_identifier; + TableExpressionNodeToColumns table_expression_node_to_columns; + size_t column_identifier_counter = 0; + + ActionsChain actions_chain; + + ActionsDAGPtr where_actions; + std::string where_action_node_name; + ActionsDAGPtr projection_actions; + + ContextPtr query_context; + + std::string getColumnUniqueIdentifier() + { + auto result = "__column_" + std::to_string(column_identifier_counter); + ++column_identifier_counter; + + return result; + } +}; struct QueryTreeActionsScopeNode { @@ -93,19 +534,19 @@ struct QueryTreeActionsScopeNode node_name_to_node[node.result_name] = &node; } - bool containsNode(const std::string & node_name) - { - return node_name_to_node.find(node_name) != node_name_to_node.end(); - } + // bool containsNode(const std::string & node_name) + // { + // return node_name_to_node.find(node_name) != node_name_to_node.end(); + // } - const ActionsDAG::Node * tryGetNode(const std::string & node_name) - { - auto it = node_name_to_node.find(node_name); - if (it == node_name_to_node.end()) - return {}; + // const ActionsDAG::Node * tryGetNode(const std::string & node_name) + // { + // auto it = node_name_to_node.find(node_name); + // if (it == node_name_to_node.end()) + // return {}; - return it->second; - } + // return it->second; + // } const ActionsDAG::Node * getNodeOrThrow(const std::string & node_name) { @@ -196,13 +637,11 @@ struct QueryTreeActionsScopeNode QueryTreeNodePtr scope_node; }; -class QueryTreeActionsVisitor : public WithContext +class QueryTreeActionsVisitor { public: - explicit QueryTreeActionsVisitor( - ActionsDAGPtr actions_dag, - ContextPtr context_) - : WithContext(context_) + explicit QueryTreeActionsVisitor(ActionsDAGPtr actions_dag, const PlannerContext & planner_context_) + : planner_context(planner_context_) { actions_stack.emplace_back(std::move(actions_dag), nullptr); } @@ -230,111 +669,92 @@ public: private: - std::pair visitImpl(QueryTreeNodePtr node) + using NodeNameAndNodeMinLevel = std::pair; + + NodeNameAndNodeMinLevel visitImpl(QueryTreeNodePtr node) { if (auto * column_node = node->as()) - return visitColumn(*column_node); + return visitColumn(node); else if (auto * constant_node = node->as()) - return visitConstant(*constant_node); + return visitConstant(node); else if (auto * function_node = node->as()) - return visitFunction(*function_node); + return visitFunction(node); throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected only column, constant or function node. Actual {}", node->formatASTForErrorMessage()); } - std::pair visitColumn(ColumnNode & column) + NodeNameAndNodeMinLevel visitColumn(const QueryTreeNodePtr & node) { - const auto & column_name = column.getColumnName(); + auto column_node_name = getActionsDAGNodeName(node.get()); + const auto & column_node = node->as(); Int64 actions_stack_size = static_cast(actions_stack.size() - 1); for (Int64 i = actions_stack_size; i >= 0; --i) { - actions_stack[i].addInputColumnIfNecessary(column_name, column.getColumnType()); + actions_stack[i].addInputColumnIfNecessary(column_node_name, column_node.getColumnType()); - if (column.getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA && - actions_stack[i].scope_node.get() == column.getColumnSource().get()) + if (column_node.getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA && + actions_stack[i].scope_node.get() == column_node.getColumnSource().get()) { - return {column_name, i}; + return {column_node_name, i}; } } - return {column_name, 0}; + return {column_node_name, 0}; } - std::pair visitConstant(ConstantNode & constant_node) + NodeNameAndNodeMinLevel visitConstant(const QueryTreeNodePtr & node) { + auto constant_node_name = getActionsDAGNodeName(node.get()); + const auto & constant_node = node->as(); const auto & literal = constant_node.getConstantValue(); - auto constant_name = constant_node.getName(); - ColumnWithTypeAndName column; - column.name = constant_name; + column.name = constant_node_name; column.type = constant_node.getResultType(); column.column = column.type->createColumnConst(1, literal); - actions_stack[0].addConstantIfNecessary(constant_name, column); + actions_stack[0].addConstantIfNecessary(constant_node_name, column); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputConstantColumnIfNecessary(constant_name, column); + actions_stack_node.addInputConstantColumnIfNecessary(constant_node_name, column); } - return {constant_name, 0}; + return {constant_node_name, 0}; } - std::pair visitLambda(QueryTreeNodePtr lambda_node_untyped) + NodeNameAndNodeMinLevel visitLambda(const QueryTreeNodePtr & node) { - auto & lambda_node = lambda_node_untyped->as(); + auto & lambda_node = node->as(); auto result_type = lambda_node.getResultType(); if (!result_type) throw Exception(ErrorCodes::LOGICAL_ERROR, "Lambda {} is not resolved during query analysis", lambda_node.formatASTForErrorMessage()); - // std::cout << "QueryTreeActionsVisitor::visitLambda " << lambda_node.formatASTForErrorMessage() << std::endl; - // std::cout << "Lambda arguments nodes size " << lambda_node.getArguments().getNodes().size() << std::endl; - NamesAndTypesList lambda_arguments_names_and_types; for (auto & lambda_node_argument : lambda_node.getArguments().getNodes()) { auto lambda_argument_name = lambda_node_argument->getName(); auto lambda_argument_type = lambda_node_argument->getResultType(); - // std::cout << "Lambda argument name " << lambda_argument_name; - // std::cout << " type " << lambda_argument_type->getName() << std::endl; lambda_arguments_names_and_types.emplace_back(lambda_argument_name, lambda_argument_type); } size_t previous_scope_node_actions_stack_index = actions_stack.size() - 1; auto lambda_actions_dag = std::make_shared(); - actions_stack.emplace_back(lambda_actions_dag, lambda_node_untyped); + actions_stack.emplace_back(lambda_actions_dag, node); - auto [node_name, level] = visitImpl(lambda_node.getExpression()); - auto lambda_result_node_name = node_name; - lambda_actions_dag->getIndex().push_back(actions_stack.back().getNodeOrThrow(node_name)); - - // std::cout << "Previous DAG nodes " << actions_stack[previous_scope_node_actions_stack_index].actions_dag.get() << std::endl; - // for (const auto & previous_actions_node : actions_stack[previous_scope_node_actions_stack_index].actions_dag->getNodes()) - // { - // std::cout << "Node " << &previous_actions_node << " result name " << previous_actions_node.result_name << std::endl; - // std::cout << "Children " << previous_actions_node.children.size() << std::endl; - // for (const auto * child_node : previous_actions_node.children) - // { - // std::cout << "Child node " << child_node << " result name " << child_node->result_name << std::endl; - // } - // } - - lambda_actions_dag->removeUnusedActions(Names(1, lambda_result_node_name)); - - // std::cout << "Lambda actions DAG Node " << node_name << " level " << level << std::endl; - // std::cout << "Lambda actions DAG " << lambda_actions_dag.get() << std::endl; - // std::cout << lambda_actions_dag->dumpDAG() << std::endl; + auto [lambda_expression_node_name, level] = visitImpl(lambda_node.getExpression()); + lambda_actions_dag->getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name)); + lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name)); auto lambda_actions = std::make_shared( - lambda_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + lambda_actions_dag, ExpressionActionsSettings::fromContext(planner_context.query_context, CompileExpressions::yes)); Names captured_column_names; ActionsDAG::NodeRawConstPtrs lambda_children; @@ -354,31 +774,30 @@ private: } } - auto lambda_hash = lambda_node.getTreeHash(); - std::string lambda_name = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); - + auto lambda_node_name = getActionsDAGNodeName(node.get()); auto function_capture = std::make_shared( - lambda_actions, captured_column_names, lambda_arguments_names_and_types, result_type, lambda_result_node_name); + lambda_actions, captured_column_names, lambda_arguments_names_and_types, result_type, lambda_expression_node_name); actions_stack.pop_back(); if (level == actions_stack.size()) --level; - actions_stack[level].addFunctionIfNecessary(lambda_name, lambda_children, function_capture); + actions_stack[level].addFunctionIfNecessary(lambda_node_name, lambda_children, function_capture); size_t actions_stack_size = actions_stack.size(); for (size_t i = level + 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputColumnIfNecessary(lambda_name, result_type); + actions_stack_node.addInputColumnIfNecessary(lambda_node_name, result_type); } - return {lambda_name, level}; + return {lambda_node_name, level}; } - std::pair visitFunction(FunctionNode & function_node) + NodeNameAndNodeMinLevel visitFunction(const QueryTreeNodePtr & node) { - auto function_node_name = function_node.getName(); + auto function_node_name = getActionsDAGNodeName(node.get()); + const auto & function_node = node->as(); if (function_node.getFunctionName() == "grouping") { @@ -409,14 +828,14 @@ private: return {function_node_name, 0}; } - auto & function_arguments = function_node.getArguments().getNodes(); + const auto & function_arguments = function_node.getArguments().getNodes(); size_t function_arguments_size = function_arguments.size(); Names function_arguments_node_names; function_arguments_node_names.reserve(function_arguments_size); size_t level = 0; - for (auto & argument : function_arguments) + for (const auto & argument : function_arguments) { if (argument->getNodeType() == QueryTreeNodeType::LAMBDA) { @@ -461,7 +880,98 @@ private: return {function_node_name, level}; } + String getActionsDAGNodeName(const IQueryTreeNode * node) const + { + String result; + auto node_type = node->getNodeType(); + + switch (node_type) + { + case QueryTreeNodeType::COLUMN: + { + auto it = planner_context.table_expression_column_node_to_column_identifier.find(node); + if (it == planner_context.table_expression_column_node_to_column_identifier.end()) + return node->getName(); + + result = it->second; + break; + } + case QueryTreeNodeType::CONSTANT: + { + result = "__constant_" + node->getName(); + break; + } + case QueryTreeNodeType::FUNCTION: + { + const auto & function_node = node->as(); + + WriteBufferFromOwnString buffer; + buffer << function_node.getFunctionName(); + + const auto & function_parameters_nodes = function_node.getParameters().getNodes(); + + if (!function_parameters_nodes.empty()) + { + buffer << '('; + + size_t function_parameters_nodes_size = function_parameters_nodes.size(); + for (size_t i = 0; i < function_parameters_nodes_size; ++i) + { + const auto & function_parameter_node = function_parameters_nodes[i]; + getActionsDAGNodeName(function_parameter_node.get()); + + if (i + 1 != function_parameters_nodes_size) + buffer << ", "; + } + + buffer << ')'; + } + + const auto & function_arguments_nodes = function_node.getArguments().getNodes(); + + buffer << '('; + + size_t function_arguments_nodes_size = function_arguments_nodes.size(); + for (size_t i = 0; i < function_arguments_nodes_size; ++i) + { + const auto & function_argument_node = function_arguments_nodes[i]; + buffer << getActionsDAGNodeName(function_argument_node.get()); + + if (i + 1 != function_arguments_nodes_size) + buffer << ", "; + } + + buffer << ')'; + + result = buffer.str(); + break; + } + case QueryTreeNodeType::QUERY: + { + auto query_hash = node->getTreeHash(); + + result = "__subquery_" + std::to_string(query_hash.first) + '_' + std::to_string(query_hash.second); + break; + } + case QueryTreeNodeType::LAMBDA: + { + auto lambda_hash = node->getTreeHash(); + + result = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); + break; + } + default: + { + result = node->getName(); + break; + } + } + + return result; + } + std::vector actions_stack; + const PlannerContext & planner_context; }; class CollectSourceColumnsMatcher @@ -471,7 +981,7 @@ public: struct Data { - NameSet source_columns_set; + PlannerContext & planner_context; }; static void visit(QueryTreeNodePtr & node, Data & data) @@ -480,29 +990,803 @@ public: if (!column_node) return; - if (column_node->getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA) + auto column_source_node = column_node->getColumnSource(); + auto column_source_node_type = column_source_node->getNodeType(); + + if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "ARRAY JOIN is not supported"); + + if (column_source_node_type == QueryTreeNodeType::LAMBDA) return; - /// Replace ALIAS column with expression - - if (column_node->hasAliasExpression()) + if (column_node->hasExpression()) { - node = column_node->getAliasExpression(); + /// Replace ALIAS column with expression + node = column_node->getExpression(); visit(node, data); return; } - data.source_columns_set.insert(column_node->getColumnName()); + if (column_source_node_type != QueryTreeNodeType::TABLE && + column_source_node_type != QueryTreeNodeType::TABLE_FUNCTION && + column_source_node_type != QueryTreeNodeType::QUERY) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected table, table function or query column source. Actual {}", + column_source_node->formatASTForErrorMessage()); + + auto & table_expression_node_to_columns = data.planner_context.table_expression_node_to_columns; + auto & table_expression_column_node_to_column_identifier = data.planner_context.table_expression_column_node_to_column_identifier; + + auto [it, _] = table_expression_node_to_columns.emplace(column_source_node.get(), TableExpressionColumns()); + auto [source_columns_set_it, inserted] = it->second.source_columns_set.insert(column_node->getColumnName()); + + if (inserted) + { + auto column_identifier = data.planner_context.getColumnUniqueIdentifier(); + table_expression_column_node_to_column_identifier.emplace(column_node, column_identifier); + it->second.column_name_to_column_identifier.emplace(column_node->getColumnName(), column_identifier); + it->second.source_input_columns.emplace_back(column_node->getColumn()); + } + else + { + auto column_identifier_it = it->second.column_name_to_column_identifier.find(column_node->getColumnName()); + if (column_identifier_it == it->second.column_name_to_column_identifier.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column node {} column identifier is not initialized", + column_node->formatASTForErrorMessage()); + + table_expression_column_node_to_column_identifier.emplace(column_node, column_identifier_it->second); + } } - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { - return true; + return child_node->getNodeType() != QueryTreeNodeType::QUERY; } }; using CollectSourceColumnsVisitor = CollectSourceColumnsMatcher::Visitor; +ActionsDAGPtr convertExpressionNodeIntoDAG(const QueryTreeNodePtr & expression_node, const ColumnsWithTypeAndName & inputs, const PlannerContext & planner_context) +{ + ActionsDAGPtr action_dag = std::make_shared(inputs); + QueryTreeActionsVisitor actions_visitor(action_dag, planner_context); + auto expression_dag_index_nodes = actions_visitor.visit(expression_node); + action_dag->getOutputs().clear(); + + for (auto & expression_dag_index_node : expression_dag_index_nodes) + action_dag->getOutputs().push_back(expression_dag_index_node); + + return action_dag; +} + +struct JoinTreeNodePlan +{ + QueryPlanBuilder plan_builder; + std::vector actions_chain_node_indices; +}; + +JoinTreeNodePlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContext & planner_context); + +JoinTreeNodePlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, + SelectQueryInfo & table_expression_query_info, + const SelectQueryOptions & select_query_options, + PlannerContext & planner_context) +{ + auto * table_node = table_expression->as(); + auto * table_function_node = table_expression->as(); + auto * query_node = table_expression->as(); + + QueryPlan query_plan; + + /** Use default columns to support case when there are no columns in query. + * Example: SELECT 1; + */ + const auto & [it, _] = planner_context.table_expression_node_to_columns.emplace(table_expression.get(), TableExpressionColumns()); + auto & table_expression_columns = it->second; + + if (table_node || table_function_node) + { + const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + + auto from_stage = storage->getQueryProcessingStage(planner_context.query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); + + Names column_names(table_expression_columns.source_columns_set.begin(), table_expression_columns.source_columns_set.end()); + + std::optional read_additional_column; + + bool plan_has_multiple_table_expressions = planner_context.table_expression_node_to_columns.size() > 1; + if (column_names.empty() && (plan_has_multiple_table_expressions || storage->getName() == "SystemOne")) + { + auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); + read_additional_column = column_names_and_types.front(); + } + + if (read_additional_column) + { + column_names.push_back(read_additional_column->name); + table_expression_columns.source_columns_set.emplace(read_additional_column->name); + table_expression_columns.source_input_columns.emplace_back(*read_additional_column); + table_expression_columns.column_name_to_column_identifier.emplace(read_additional_column->name, planner_context.getColumnUniqueIdentifier()); + } + + if (!column_names.empty()) + { + size_t max_block_size = planner_context.query_context->getSettingsRef().max_block_size; + size_t max_streams = planner_context.query_context->getSettingsRef().max_threads; + storage->read(query_plan, column_names, storage_snapshot, table_expression_query_info, planner_context.query_context, from_stage, max_block_size, max_streams); + } + + /// Create step which reads from empty source if storage has no data. + if (!query_plan.isInitialized()) + { + auto source_header = storage_snapshot->getSampleBlockForColumns(column_names); + Pipe pipe(std::make_shared(source_header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource"); + query_plan.addStep(std::move(read_from_pipe)); + } + } + else if (query_node) + { + InterpreterSelectQueryAnalyzer interpeter(table_expression, select_query_options, planner_context.query_context); + interpeter.initializeQueryPlanIfNeeded(); + query_plan = std::move(interpeter).extractQueryPlan(); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected table, table function or query. Actual {}", table_expression->formatASTForErrorMessage()); + } + + auto rename_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + + for (const auto & [column_name, column_identifier] : table_expression_columns.column_name_to_column_identifier) + { + auto position = query_plan.getCurrentDataStream().header.getPositionByName(column_name); + const auto * node_to_rename = rename_actions_dag->getOutputs()[position]; + rename_actions_dag->getOutputs()[position] = &rename_actions_dag->addAlias(*node_to_rename, column_identifier); + } + + planner_context.actions_chain.addNode(std::make_unique(rename_actions_dag, true /*available_output_columns_only_aliases*/)); + size_t actions_chain_node_index = planner_context.actions_chain.getLastNodeIndex(); + + QueryPlanBuilder builder(std::move(query_plan)); + + builder.addBuildStep([rename_actions_dag](QueryPlan & build_plan) { + auto rename_step = std::make_unique(build_plan.getCurrentDataStream(), rename_actions_dag); + rename_step->setStepDescription("Change column names to column identifiers"); + build_plan.addStep(std::move(rename_step)); + }); + + return {std::move(builder), {actions_chain_node_index}}; +} + +class JoinClause +{ +public: + void addKey(const ActionsDAG::Node * left_key_node, const ActionsDAG::Node * right_key_node) + { + left_key_nodes.emplace_back(left_key_node); + right_key_nodes.emplace_back(right_key_node); + } + + void addCondition(JoinTableSide table_side, const ActionsDAG::Node * condition_node) + { + auto & filter_condition_nodes = table_side == JoinTableSide::Left ? left_filter_condition_nodes : right_filter_condition_nodes; + filter_condition_nodes.push_back(condition_node); + } + + const ActionsDAG::NodeRawConstPtrs & getLeftKeyNodes() const + { + return left_key_nodes; + } + + const ActionsDAG::NodeRawConstPtrs & getRightKeyNodes() const + { + return right_key_nodes; + } + + const ActionsDAG::NodeRawConstPtrs & getLeftFilterConditionNodes() const + { + return left_filter_condition_nodes; + } + + const ActionsDAG::NodeRawConstPtrs & getRightFilterConditionNodes() const + { + return right_filter_condition_nodes; + } + + void clearConditionNodes(JoinTableSide table_side) + { + auto & filter_condition_nodes = table_side == JoinTableSide::Left ? left_filter_condition_nodes : right_filter_condition_nodes; + filter_condition_nodes.clear(); + } + + void dump(WriteBuffer & buffer) const + { + auto dump_dag_nodes = [&](const ActionsDAG::NodeRawConstPtrs & dag_nodes) + { + String dag_nodes_dump; + + if (!dag_nodes.empty()) + { + for (const auto & dag_node : dag_nodes) + { + dag_nodes_dump += dag_node->result_name; + dag_nodes_dump += ", "; + } + + dag_nodes_dump.pop_back(); + dag_nodes_dump.pop_back(); + } + + return dag_nodes_dump; + }; + + buffer << "left_key_nodes: " << dump_dag_nodes(left_key_nodes); + buffer << " right_key_nodes: " << dump_dag_nodes(right_key_nodes); + + if (!left_filter_condition_nodes.empty()) + buffer << " left_condition_nodes: " + dump_dag_nodes(left_filter_condition_nodes); + + if (!right_filter_condition_nodes.empty()) + buffer << " left_condition_nodes: " + dump_dag_nodes(right_filter_condition_nodes); + } + + [[maybe_unused]] String dump() const + { + WriteBufferFromOwnString buffer; + dump(buffer); + + return buffer.str(); + } +private: + ActionsDAG::NodeRawConstPtrs left_key_nodes; + ActionsDAG::NodeRawConstPtrs right_key_nodes; + + ActionsDAG::NodeRawConstPtrs left_filter_condition_nodes; + ActionsDAG::NodeRawConstPtrs right_filter_condition_nodes; +}; + +using JoinClauses = std::vector; + +std::optional extractJoinTableSideFromExpression(const ActionsDAG::Node * expression_root_node, + const NameSet & left_table_expression_columns_names, + const NameSet & right_table_expression_columns_names, + const JoinNode & join_node) +{ + std::optional table_side; + std::vector nodes_to_process; + nodes_to_process.push_back(expression_root_node); + + while (!nodes_to_process.empty()) + { + const auto * node_to_process = nodes_to_process.back(); + nodes_to_process.pop_back(); + + for (const auto & child : node_to_process->children) + nodes_to_process.push_back(child); + + if (node_to_process->type != ActionsDAG::ActionType::INPUT) + continue; + + const auto & input_name = node_to_process->result_name; + + bool left_table_expression_contains_input = left_table_expression_columns_names.contains(input_name); + bool right_table_expression_contains_input = right_table_expression_columns_names.contains(input_name); + + if (!left_table_expression_contains_input && !right_table_expression_contains_input) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} actions has column {} that do not exist in left {} or right {} table expression columns", + join_node.formatASTForErrorMessage(), + input_name, + boost::join(left_table_expression_columns_names, ", "), + boost::join(right_table_expression_columns_names, ", ")); + + auto input_table_side = left_table_expression_contains_input ? JoinTableSide::Left : JoinTableSide::Right; + if (table_side && (*table_side) != input_table_side) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} join expression contains column from left and right table", + join_node.formatASTForErrorMessage()); + + table_side = input_table_side; + } + + return table_side; +} + +void buildJoinClause(ActionsDAGPtr join_expression_dag, + const ActionsDAG::Node * join_expressions_actions_node, + const NameSet & left_table_expression_columns_names, + const NameSet & right_table_expression_columns_names, + const JoinNode & join_node, + JoinClause & join_clause) +{ + /// For and function go into children + if (join_expressions_actions_node->function && join_expressions_actions_node->function->getName() == "and") + { + for (const auto & child : join_expressions_actions_node->children) + { + buildJoinClause(join_expression_dag, + child, + left_table_expression_columns_names, + right_table_expression_columns_names, + join_node, + join_clause); + } + + return; + } + + if (join_expressions_actions_node->function && join_expressions_actions_node->function->getName() == "equals") + { + const auto * equals_left_child = join_expressions_actions_node->children.at(0); + const auto * equals_right_child = join_expressions_actions_node->children.at(1); + + auto left_equals_expression_side_optional = extractJoinTableSideFromExpression(equals_left_child, + left_table_expression_columns_names, + right_table_expression_columns_names, + join_node); + + auto right_equals_expression_side_optional = extractJoinTableSideFromExpression(equals_right_child, + left_table_expression_columns_names, + right_table_expression_columns_names, + join_node); + + if (!left_equals_expression_side_optional && !right_equals_expression_side_optional) + { + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} ON expression {} with constants is not supported", + join_node.formatASTForErrorMessage(), + join_expressions_actions_node->function->getName()); + } + else if (left_equals_expression_side_optional && !right_equals_expression_side_optional) + { + join_clause.addCondition(*left_equals_expression_side_optional, join_expressions_actions_node); + } + else if (!left_equals_expression_side_optional && right_equals_expression_side_optional) + { + join_clause.addCondition(*right_equals_expression_side_optional, join_expressions_actions_node); + } + else + { + auto left_equals_expression_side = *left_equals_expression_side_optional; + auto right_equals_expression_side = *right_equals_expression_side_optional; + + if (left_equals_expression_side != right_equals_expression_side) + join_clause.addKey(equals_left_child, equals_right_child); + else + join_clause.addCondition(left_equals_expression_side, join_expressions_actions_node); + } + + return; + } + + auto expression_side_optional = extractJoinTableSideFromExpression(join_expressions_actions_node, + left_table_expression_columns_names, + right_table_expression_columns_names, + join_node); + + if (!expression_side_optional) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} with constants is not supported", + join_node.formatASTForErrorMessage()); + + auto expression_side = *expression_side_optional; + + join_clause.addCondition(expression_side, join_expressions_actions_node); +} + +struct JoinClausesAndActions +{ + JoinClauses join_clauses; + ActionsDAGPtr join_expression_actions; + ActionsDAGPtr left_join_expressions_actions; + ActionsDAGPtr right_join_expressions_actions; +}; + +JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & join_expression_input_columns, + const ColumnsWithTypeAndName & left_table_expression_columns, + const ColumnsWithTypeAndName & right_table_expression_columns, + const JoinNode & join_node, + const PlannerContext & planner_context) +{ + std::cout << "buildJoinClausesAndActions " << join_node.formatASTForErrorMessage() << std::endl; + + ActionsDAGPtr join_expression_actions = std::make_shared(join_expression_input_columns); + // std::cout << "buildJoinClausesAndActions join expression actions dag before visitor " << std::endl; + // std::cout << join_expression_actions->dumpDAG() << std::endl; + + QueryTreeActionsVisitor join_expression_visitor(join_expression_actions, planner_context); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_node.getJoinExpression()); + if (join_expression_dag_node_raw_pointers.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} ON clause contains multiple expressions", + join_node.formatASTForErrorMessage()); + + // std::cout << "buildJoinClausesAndActions join expression actions dag after visitor " << std::endl; + // std::cout << join_expression_actions->dumpDAG() << std::endl; + + const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0]; + if (!join_expressions_actions_root_node->function) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} join expression expected function", + join_node.formatASTForErrorMessage()); + + std::cout << "buildJoinClausesAndActions join expressions actions DAG dump " << std::endl; + std::cout << join_expression_actions->dumpDAG() << std::endl; + + std::cout << "root node " << join_expressions_actions_root_node << std::endl; + + size_t left_table_expression_columns_size = left_table_expression_columns.size(); + + Names join_left_actions_names; + join_left_actions_names.reserve(left_table_expression_columns_size); + + NameSet join_left_actions_names_set; + join_left_actions_names_set.reserve(left_table_expression_columns_size); + + for (const auto & left_table_expression_column : left_table_expression_columns) + { + join_left_actions_names.push_back(left_table_expression_column.name); + join_left_actions_names_set.insert(left_table_expression_column.name); + } + + size_t right_table_expression_columns_size = right_table_expression_columns.size(); + + Names join_right_actions_names; + join_right_actions_names.reserve(right_table_expression_columns_size); + + NameSet join_right_actions_names_set; + join_right_actions_names_set.reserve(right_table_expression_columns_size); + + for (const auto & right_table_expression_column : right_table_expression_columns) + { + join_right_actions_names.push_back(right_table_expression_column.name); + join_right_actions_names_set.insert(right_table_expression_column.name); + } + + JoinClausesAndActions result; + result.join_expression_actions = join_expression_actions; + + const auto & function_name = join_expressions_actions_root_node->function->getName(); + if (function_name == "or") + { + for (const auto & child : join_expressions_actions_root_node->children) + { + result.join_clauses.emplace_back(); + + buildJoinClause(join_expression_actions, + child, + join_left_actions_names_set, + join_right_actions_names_set, + join_node, + result.join_clauses.back()); + } + } + else + { + result.join_clauses.emplace_back(); + + buildJoinClause(join_expression_actions, + join_expressions_actions_root_node, + join_left_actions_names_set, + join_right_actions_names_set, + join_node, + result.join_clauses.back()); + } + + auto and_function = FunctionFactory::instance().get("and", planner_context.query_context); + + auto add_necessary_name_if_needed = [&](JoinTableSide join_table_side, const String & name) + { + auto & necessary_names = join_table_side == JoinTableSide::Left ? join_left_actions_names : join_right_actions_names; + auto & necessary_names_set = join_table_side == JoinTableSide::Left ? join_left_actions_names_set : join_right_actions_names_set; + + auto [_, inserted] = necessary_names_set.emplace(name); + if (inserted) + necessary_names.push_back(name); + }; + + for (auto & join_clause : result.join_clauses) + { + const auto & left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); + if (!left_filter_condition_nodes.empty()) + { + const ActionsDAG::Node * dag_filter_condition_node = nullptr; + + if (left_filter_condition_nodes.size() > 1) + dag_filter_condition_node = &join_expression_actions->addFunction(and_function, left_filter_condition_nodes, {}); + else + dag_filter_condition_node = left_filter_condition_nodes[0]; + + join_clause.clearConditionNodes(JoinTableSide::Left); + join_clause.addCondition(JoinTableSide::Left, dag_filter_condition_node); + + join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node); + + add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name); + } + + const auto & right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); + if (!right_filter_condition_nodes.empty()) + { + const ActionsDAG::Node * dag_filter_condition_node = nullptr; + + if (right_filter_condition_nodes.size() > 1) + dag_filter_condition_node = &join_expression_actions->addFunction(and_function, right_filter_condition_nodes, {}); + else + dag_filter_condition_node = right_filter_condition_nodes[0]; + + join_clause.clearConditionNodes(JoinTableSide::Right); + join_clause.addCondition(JoinTableSide::Right, dag_filter_condition_node); + + join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node); + + add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name); + } + + for (const auto & left_key_node : join_clause.getLeftKeyNodes()) + { + join_expression_actions->addOrReplaceInOutputs(*left_key_node); + add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name); + } + + for (const auto & right_key_node : join_clause.getRightKeyNodes()) + { + join_expression_actions->addOrReplaceInOutputs(*right_key_node); + add_necessary_name_if_needed(JoinTableSide::Right, right_key_node->result_name); + } + } + + result.left_join_expressions_actions = join_expression_actions->clone(); + result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names); + + result.right_join_expressions_actions = join_expression_actions->clone(); + result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); + + return result; +} + +JoinTreeNodePlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContext & planner_context) +{ + auto & join_node = join_tree_node->as(); + + auto left_plan_build_result = buildQueryPlanForJoinTreeNode(join_node.getLeftTableExpression(), + select_query_info, + select_query_options, + planner_context); + auto left_plan_builder = std::move(left_plan_build_result.plan_builder); + ColumnsWithTypeAndName left_plan_output_columns = planner_context.actions_chain.getAvailableOutputColumns(left_plan_build_result.actions_chain_node_indices); + + auto right_plan_build_result = buildQueryPlanForJoinTreeNode(join_node.getRightTableExpression(), + select_query_info, + select_query_options, + planner_context); + auto right_plan_builder = std::move(right_plan_build_result.plan_builder); + auto right_plan_output_columns = planner_context.actions_chain.getAvailableOutputColumns(right_plan_build_result.actions_chain_node_indices); + + if (join_node.getStrictness() == JoinStrictness::Asof) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "JOIN {} ASOF is not supported", + join_node.formatASTForErrorMessage()); + + JoinClausesAndActions join_clauses_and_actions; + + std::vector actions_chain_node_indices; + std::vector actions_chain_right_plan_node_indexes; + + if (join_node.getJoinExpression()) + { + if (join_node.isUsingJoinExpression()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "JOIN {} USING is unsupported", + join_node.formatASTForErrorMessage()); + + auto join_expression_input_columns = left_plan_output_columns; + join_expression_input_columns.insert(join_expression_input_columns.end(), right_plan_output_columns.begin(), right_plan_output_columns.end()); + join_clauses_and_actions = buildJoinClausesAndActions(join_expression_input_columns, + left_plan_output_columns, + right_plan_output_columns, + join_node, + planner_context); + + auto left_join_actions_node = std::make_unique(join_clauses_and_actions.left_join_expressions_actions); + left_join_actions_node->addParentIndices(left_plan_build_result.actions_chain_node_indices); + planner_context.actions_chain.addNode(std::move(left_join_actions_node)); + actions_chain_node_indices.push_back(planner_context.actions_chain.getLastNodeIndex()); + + auto right_join_actions_node = std::make_unique(join_clauses_and_actions.right_join_expressions_actions); + right_join_actions_node->addParentIndices(right_plan_build_result.actions_chain_node_indices); + planner_context.actions_chain.addNode(std::move(right_join_actions_node)); + actions_chain_node_indices.push_back(planner_context.actions_chain.getLastNodeIndex()); + actions_chain_right_plan_node_indexes.push_back(planner_context.actions_chain.getLastNodeIndex()); + + left_plan_builder.addBuildStep([left_join_expressions_actions = join_clauses_and_actions.left_join_expressions_actions](QueryPlan & build_plan) + { + auto left_join_expressions_actions_step = std::make_unique(build_plan.getCurrentDataStream(), left_join_expressions_actions); + left_join_expressions_actions_step->setStepDescription("Join actions"); + build_plan.addStep(std::move(left_join_expressions_actions_step)); + }); + + right_plan_builder.addBuildStep([right_join_expressions_actions = join_clauses_and_actions.right_join_expressions_actions](QueryPlan & build_plan) + { + auto right_join_expressions_actions_step = std::make_unique(build_plan.getCurrentDataStream(), right_join_expressions_actions); + right_join_expressions_actions_step->setStepDescription("Join actions"); + build_plan.addStep(std::move(right_join_expressions_actions_step)); + }); + } + else + { + actions_chain_right_plan_node_indexes = right_plan_build_result.actions_chain_node_indices; + actions_chain_node_indices.insert(actions_chain_node_indices.end(), actions_chain_right_plan_node_indexes.begin(), actions_chain_right_plan_node_indexes.end()); + } + + std::vector builders; + builders.emplace_back(std::move(left_plan_builder)); + builders.emplace_back(std::move(right_plan_builder)); + + QueryPlanBuilder builder(std::move(builders), [join_clauses_and_actions, actions_chain_right_plan_node_indexes, &join_node, &planner_context](std::vector build_query_plans) + { + if (build_query_plans.size() != 2) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Join step expects 2 query plans. Actual {}", build_query_plans.size()); + + auto left_plan = std::move(build_query_plans[0]); + auto right_plan = std::move(build_query_plans[1]); + + auto table_join = std::make_shared(); + table_join->getTableJoin() = join_node.toASTTableJoin()->as(); + if (join_node.getKind() == JoinKind::Comma) + table_join->getTableJoin().kind = JoinKind::Cross; + table_join->getTableJoin().strictness = JoinStrictness::All; + + NameSet join_clauses_right_column_names; + + if (join_node.getJoinExpression()) + { + const auto & join_clauses = join_clauses_and_actions.join_clauses; + auto & table_join_clauses = table_join->getClauses(); + + for (const auto & join_clause : join_clauses) + { + table_join_clauses.emplace_back(); + auto & table_join_clause = table_join_clauses.back(); + + const auto & join_clause_left_key_nodes = join_clause.getLeftKeyNodes(); + const auto & join_clause_right_key_nodes = join_clause.getRightKeyNodes(); + + size_t join_clause_key_nodes_size = join_clause_left_key_nodes.size(); + assert(join_clause_key_nodes_size == join_clause_right_key_nodes.size()); + + for (size_t i = 0; i < join_clause_key_nodes_size; ++i) + { + table_join_clause.key_names_left.push_back(join_clause_left_key_nodes[i]->result_name); + table_join_clause.key_names_right.push_back(join_clause_right_key_nodes[i]->result_name); + join_clauses_right_column_names.insert(join_clause_right_key_nodes[i]->result_name); + } + + const auto & join_clause_get_left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); + if (!join_clause_get_left_filter_condition_nodes.empty()) + { + if (join_clause_get_left_filter_condition_nodes.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} left filter conditions size must be 1. Actual {}", + join_node.formatASTForErrorMessage(), + join_clause_get_left_filter_condition_nodes.size()); + + const auto & join_clause_left_filter_condition_name = join_clause_get_left_filter_condition_nodes[0]->result_name; + table_join_clause.analyzer_left_filter_condition_column_name = join_clause_left_filter_condition_name; + } + + const auto & join_clause_get_right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); + if (!join_clause_get_right_filter_condition_nodes.empty()) + { + if (join_clause_get_right_filter_condition_nodes.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} right filter conditions size must be 1. Actual {}", + join_node.formatASTForErrorMessage(), + join_clause_get_right_filter_condition_nodes.size()); + + const auto & join_clause_right_filter_condition_name = join_clause_get_right_filter_condition_nodes[0]->result_name; + table_join_clause.analyzer_right_filter_condition_column_name = join_clause_right_filter_condition_name; + join_clauses_right_column_names.insert(join_clause_right_filter_condition_name); + } + } + } + + auto left_table_names = left_plan.getCurrentDataStream().header.getNames(); + NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); + + auto columns_from_joined_table = right_plan.getCurrentDataStream().header.getNamesAndTypesList(); + table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); + + NamesAndTypesList columns_added_by_join; + for (auto & column_from_joined_table : columns_from_joined_table) + { + for (const auto & actions_chain_right_plan_node_index : actions_chain_right_plan_node_indexes) + { + const auto & child_required_ouput_columns_names = planner_context.actions_chain[actions_chain_right_plan_node_index]->getChildRequiredOutputColumnsNames(); + + if (child_required_ouput_columns_names.contains(column_from_joined_table.name)) + { + columns_added_by_join.insert(columns_added_by_join.end(), column_from_joined_table); + break; + } + } + } + + table_join->setColumnsAddedByJoin(columns_added_by_join); + + size_t max_block_size = planner_context.query_context->getSettingsRef().max_block_size; + size_t max_streams = planner_context.query_context->getSettingsRef().max_threads; + + JoinPtr join_ptr = std::make_shared(table_join, right_plan.getCurrentDataStream().header, false /*any_take_last_row*/); + QueryPlanStepPtr join_step = std::make_unique( + left_plan.getCurrentDataStream(), + right_plan.getCurrentDataStream(), + join_ptr, + max_block_size, + max_streams, + false /*optimize_read_in_order*/); + + join_step->setStepDescription(fmt::format("JOIN {}", JoinPipelineType::FillRightFirst)); + + std::vector plans; + plans.emplace_back(std::make_unique(std::move(left_plan))); + plans.emplace_back(std::make_unique(std::move(right_plan))); + + auto result = QueryPlan(); + result.unitePlans(std::move(join_step), {std::move(plans)}); + + return result; + }); + + return {std::move(builder), actions_chain_node_indices}; +} + +JoinTreeNodePlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContext & planner_context) +{ + auto join_tree_node_type = join_tree_node->getNodeType(); + + switch (join_tree_node_type) + { + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + SelectQueryInfo table_expression_query_info = select_query_info; + return buildQueryPlanForTableExpression(join_tree_node, table_expression_query_info, select_query_options, planner_context); + } + case QueryTreeNodeType::JOIN: + { + return buildQueryPlanForJoinNode(join_tree_node, select_query_info, select_query_options, planner_context); + } + case QueryTreeNodeType::ARRAY_JOIN: + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ARRAY JOIN is not supported"); + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected query, table, table function, join or array join query node. Actual {}", join_tree_node->formatASTForErrorMessage()); + } + } +} + +} + InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( const ASTPtr & query_, const SelectQueryOptions & select_query_options_, @@ -586,124 +1870,102 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() if (query_plan.isInitialized()) return; - auto & query_tree_typed = query_tree->as(); + auto & query_node = query_tree->as(); - ActionsDAGPtr action_dag = std::make_shared(); - ColumnsWithTypeAndName inputs; + auto current_context = getContext(); - CollectSourceColumnsVisitor::Data data; + SelectQueryInfo select_query_info; + select_query_info.original_query = query; + select_query_info.query = query; + + PlannerContext planner_context; + planner_context.query_context = getContext(); + + CollectSourceColumnsVisitor::Data data {planner_context}; CollectSourceColumnsVisitor collect_source_columns_visitor(data); collect_source_columns_visitor.visit(query_tree); - NameSet source_columns_set = std::move(data.source_columns_set); + JoinTreeNodePlan join_tree_node_plan = buildQueryPlanForJoinTreeNode(query_node.getFrom(), select_query_info, select_query_options, planner_context); + auto query_plan_builder = std::move(join_tree_node_plan.plan_builder); + auto action_chain_node_parent_indices = join_tree_node_plan.actions_chain_node_indices; - // std::cout << "DAG before " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl; - // std::cout << action_dag->dumpDAG() << std::endl; + if (query_node.hasWhere()) + { + ColumnsWithTypeAndName where_input = planner_context.actions_chain.getAvailableOutputColumns(action_chain_node_parent_indices); + planner_context.where_actions = convertExpressionNodeIntoDAG(query_node.getWhere(), where_input, planner_context); + planner_context.where_action_node_name = planner_context.where_actions->getOutputs().at(0)->result_name; - QueryTreeActionsVisitor visitor(action_dag, getContext()); - auto projection_action_dag_nodes = visitor.visit(query_tree_typed.getProjectionNode()); + auto where_actions_node = std::make_unique(planner_context.where_actions); + where_actions_node->addParentIndices(action_chain_node_parent_indices); + planner_context.actions_chain.addNode(std::move(where_actions_node)); + action_chain_node_parent_indices = {planner_context.actions_chain.getLastNodeIndex()}; + + size_t where_node_index = planner_context.actions_chain.size(); + + query_plan_builder.addBuildStep([&, where_node_index](QueryPlan & build_plan) + { + bool remove_filter = !planner_context.actions_chain.at(where_node_index)->getChildRequiredOutputColumnsNames().contains(planner_context.where_action_node_name); + auto where_step = std::make_unique(build_plan.getCurrentDataStream(), + planner_context.where_actions, + planner_context.where_action_node_name, + remove_filter); + where_step->setStepDescription("WHERE"); + build_plan.addStep(std::move(where_step)); + }); + } + + ColumnsWithTypeAndName projection_input = planner_context.actions_chain.getAvailableOutputColumns(action_chain_node_parent_indices); + planner_context.projection_actions = convertExpressionNodeIntoDAG(query_node.getProjectionNode(), projection_input, planner_context); + + auto projection_actions_node = std::make_unique(planner_context.projection_actions); + projection_actions_node->addParentIndices(action_chain_node_parent_indices); + planner_context.actions_chain.addNode(std::move(projection_actions_node)); + + const auto & projection_action_dag_nodes = planner_context.projection_actions->getOutputs(); size_t projection_action_dag_nodes_size = projection_action_dag_nodes.size(); - // std::cout << "Projection action dag nodes size " << projection_action_dag_nodes_size << std::endl; - // for (size_t i = 0; i < projection_action_dag_nodes_size; ++i) - // { - // std::cout << "DAG node " << projection_action_dag_nodes[i] << std::endl; - // } - - // std::cout << "DAG after " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl; - // std::cout << action_dag->dumpDAG() << std::endl; - - auto & projection_nodes = query_tree_typed.getProjection().getNodes(); + auto & projection_nodes = query_node.getProjection().getNodes(); size_t projection_nodes_size = projection_nodes.size(); if (projection_nodes_size != projection_action_dag_nodes_size) - throw Exception( - ErrorCodes::LOGICAL_ERROR, + throw Exception(ErrorCodes::LOGICAL_ERROR, "QueryTree projection nodes size mismatch. Expected {}. Actual {}", projection_action_dag_nodes_size, projection_nodes_size); NamesWithAliases projection_names; + for (size_t i = 0; i < projection_nodes_size; ++i) { auto & node = projection_nodes[i]; + auto node_name = node->getName(); const auto * action_dag_node = projection_action_dag_nodes[i]; - auto action_dag_node_name = action_dag_node->result_name; - - action_dag->getIndex().push_back(action_dag_node); + const auto & actions_dag_node_name = action_dag_node->result_name; if (node->hasAlias()) - projection_names.push_back({action_dag_node_name, node->getAlias()}); + projection_names.push_back({actions_dag_node_name, node->getAlias()}); else - projection_names.push_back({action_dag_node_name, action_dag_node_name}); + projection_names.push_back({actions_dag_node_name, node_name}); } - action_dag->project(projection_names); + planner_context.projection_actions->project(projection_names); - // std::cout << "Final DAG " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl; - // std::cout << action_dag->dumpDAG() << std::endl; - // std::cout << "Names " << action_dag->dumpNames() << std::endl; - // std::cout << "Final DAG nodes " << std::endl; - // for (const auto & node : action_dag->getNodes()) - // { - // std::cout << "Node " << &node << " result name " << node.result_name << std::endl; - // } - - // std::cout << "Source columns " << source_columns_set.size() << std::endl; - // for (const auto & source_column : source_columns_set) - // std::cout << source_column << std::endl; - - auto current_context = getContext(); - size_t max_block_size = current_context->getSettingsRef().max_block_size; - size_t max_streams = current_context->getSettingsRef().max_threads; - - SelectQueryInfo query_info; - query_info.original_query = query; - query_info.query = query; - - auto * table_node = query_tree_typed.getFrom()->as(); - auto * table_function_node = query_tree_typed.getFrom()->as(); - auto * query_node = query_tree_typed.getFrom()->as(); - - if (table_node || table_function_node) + query_plan_builder.addBuildStep([&](QueryPlan & build_plan) { - const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); - const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + auto projection_step = std::make_unique(build_plan.getCurrentDataStream(), planner_context.projection_actions); + projection_step->setStepDescription("Projection"); + build_plan.addStep(std::move(projection_step)); + }); - auto from_stage = storage->getQueryProcessingStage(current_context, select_query_options.to_stage, storage_snapshot, query_info); + // std::cout << "Chain dump before finalize" << std::endl; + // std::cout << planner_context.actions_chain.dump() << std::endl; - Names column_names(source_columns_set.begin(), source_columns_set.end()); + planner_context.actions_chain.finalize(); - if (column_names.empty() && storage->getName() == "SystemOne") - column_names.push_back("dummy"); + // std::cout << "Chain dump after finalize" << std::endl; + // std::cout << planner_context.actions_chain.dump() << std::endl; - if (!column_names.empty()) - storage->read(query_plan, column_names, storage_snapshot, query_info, getContext(), from_stage, max_block_size, max_streams); - - /// Create step which reads from empty source if storage has no data. - if (!query_plan.isInitialized()) - { - auto source_header = storage_snapshot->getSampleBlockForColumns(column_names); - Pipe pipe(std::make_shared(source_header)); - auto read_from_pipe = std::make_unique(std::move(pipe)); - read_from_pipe->setStepDescription("Read from NullSource"); - query_plan.addStep(std::move(read_from_pipe)); - } - } - else if (query_node) - { - InterpreterSelectQueryAnalyzer interpeter(query_tree_typed.getFrom(), select_query_options, getContext()); - interpeter.initializeQueryPlanIfNeeded(); - query_plan = std::move(interpeter.query_plan); - } - else - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only single table or query in FROM section are supported"); - } - - auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), action_dag); - projection_step->setStepDescription("Projection"); - query_plan.addStep(std::move(projection_step)); + query_plan = std::move(query_plan_builder).buildPlan(); } } diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 561d4054f4e..4ba01c26bdd 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -31,9 +31,14 @@ public: bool supportsTransactions() const override { return true; } -private: void initializeQueryPlanIfNeeded(); + QueryPlan && extractQueryPlan() && + { + return std::move(query_plan); + } + +private: ASTPtr query; QueryTreeNodePtr query_tree; QueryPlan query_plan; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 5d065e564b2..7461c53a342 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -46,51 +46,49 @@ namespace std::string formatTypeMap(const TableJoin::NameToTypeMap & target, const TableJoin::NameToTypeMap & source) { std::vector text; - for (const auto & [k, v] : target) + for (const auto & [key, value] : target) { - auto src_type_it = source.find(k); + auto src_type_it = source.find(key); std::string src_type_name = src_type_it != source.end() ? src_type_it->second->getName() : ""; - text.push_back(fmt::format("{} : {} -> {}", k, src_type_name, v->getName())); + text.push_back(fmt::format("{} : {} -> {}", key, src_type_name, value->getName())); } + return fmt::format("{}", fmt::join(text, ", ")); } -} - -namespace -{ - struct BothSidesTag {}; struct LeftSideTag {}; struct RightSideTag {}; -template -bool forAllKeys(OnExpr & expressions, Func callback) +template +bool forKeyNamesInJoinClauses(JoinClauses & join_clauses, Func callback) { static_assert(std::is_same_v || std::is_same_v || std::is_same_v); - for (auto & expr : expressions) + for (auto & join_clause : join_clauses) { if constexpr (std::is_same_v) - assert(expr.key_names_left.size() == expr.key_names_right.size()); + assert(join_clause.key_names_left.size() == join_clause.key_names_right.size()); - size_t sz = !std::is_same_v ? expr.key_names_left.size() : expr.key_names_right.size(); - for (size_t i = 0; i < sz; ++i) + size_t key_names_size = !std::is_same_v ? join_clause.key_names_left.size() : join_clause.key_names_right.size(); + + for (size_t i = 0; i < key_names_size; ++i) { - bool cont; + bool should_continue; if constexpr (std::is_same_v) - cont = callback(expr.key_names_left[i], expr.key_names_right[i]); - if constexpr (std::is_same_v) - cont = callback(expr.key_names_left[i]); + should_continue = callback(join_clause.key_names_left[i], join_clause.key_names_right[i]); + else if constexpr (std::is_same_v) + should_continue = callback(join_clause.key_names_left[i]); if constexpr (std::is_same_v) - cont = callback(expr.key_names_right[i]); + should_continue = callback(join_clause.key_names_right[i]); - if (!cont) + if (!should_continue) return false; } } + return true; } @@ -133,6 +131,7 @@ void TableJoin::resetCollected() void TableJoin::addUsingKey(const ASTPtr & ast) { + // std::cout << "TableJoin::addUsingKey " << ast->formatForErrorMessage() << std::endl; addKey(ast->getColumnName(), renamedRightColumnName(ast->getAliasOrColumnName()), ast); } @@ -237,6 +236,12 @@ ASTPtr TableJoin::rightKeysList() const return keys_list; } +void TableJoin::setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix) +{ + columns_from_joined_table = std::move(columns_from_joined_table_value); + deduplicateAndQualifyColumnNames(left_table_columns, right_table_prefix); +} + Names TableJoin::requiredJoinedNames() const { Names key_names_right = getAllNames(JoinTableSide::Right); @@ -256,16 +261,19 @@ Names TableJoin::requiredJoinedNames() const NameSet TableJoin::requiredRightKeys() const { - NameSet required; - forAllKeys(clauses, [this, &required](const auto & name) + NameSet required_right_column_names; + forKeyNamesInJoinClauses(clauses, [&](const auto & right_column_name) { - auto rename = renamedRightColumnName(name); + auto renamed_right_column = renamedRightColumnName(right_column_name); + for (const auto & column : columns_added_by_join) - if (rename == column.name) - required.insert(name); + if (renamed_right_column == column.name) + required_right_column_names.insert(renamed_right_column); + return true; }); - return required; + + return required_right_column_names; } NamesWithAliases TableJoin::getRequiredColumns(const Block & sample, const Names & action_required_columns) const @@ -286,7 +294,7 @@ Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vecto if (required_keys.empty()) return required_right_keys; - forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) + forKeyNamesInJoinClauses(clauses, [&](const auto & left_key_name, const auto & right_key_name) { if (required_keys.contains(right_key_name) && !required_right_keys.has(right_key_name)) { @@ -294,8 +302,10 @@ Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vecto required_right_keys.insert(right_key); keys_sources.push_back(left_key_name); } + return true; }); + return required_right_keys; } @@ -311,9 +321,15 @@ bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column) { + // std::cout << "TableJoin::addJoinedColumn " << joined_column.dump() << std::endl; columns_added_by_join.emplace_back(joined_column); } +void TableJoin::setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value) +{ + columns_added_by_join = columns_added_by_join_value; +} + NamesAndTypesList TableJoin::correctedColumnsAddedByJoin() const { NamesAndTypesList result; @@ -416,6 +432,75 @@ bool TableJoin::needStreamWithNonJoinedRows() const return isRightOrFull(kind()); } +static std::optional getDictKeyName(const String & dict_name , ContextPtr context) +{ + auto dictionary = context->getExternalDictionariesLoader().getDictionary(dict_name, context); + if (!dictionary) + return {}; + + if (const auto & structure = dictionary->getStructure(); structure.id) + return structure.id->name; + return {}; +} + +bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context) +{ + bool allowed_inner = isInner(kind()) && strictness() == JoinStrictness::All; + bool allowed_left = isLeft(kind()) && (strictness() == JoinStrictness::Any || + strictness() == JoinStrictness::All || + strictness() == JoinStrictness::Semi || + strictness() == JoinStrictness::Anti); + + /// Support ALL INNER, [ANY | ALL | SEMI | ANTI] LEFT + if (!allowed_inner && !allowed_left) + return false; + + if (clauses.size() != 1 || clauses[0].key_names_right.size() != 1) + return false; + + const auto & right_key = getOnlyClause().key_names_right[0]; + + /// TODO: support 'JOIN ... ON expr(dict_key) = table_key' + auto it_key = original_names.find(right_key); + if (it_key == original_names.end()) + return false; + + if (!right_storage_dictionary) + return false; + + auto dict_name = right_storage_dictionary->getDictionaryName(); + + auto dict_key = getDictKeyName(dict_name, context); + if (!dict_key.has_value() || *dict_key != it_key->second) + return false; /// JOIN key != Dictionary key + + Names src_names; + NamesAndTypesList dst_columns; + for (const auto & col : sample_block) + { + if (col.name == right_key) + continue; /// do not extract key column + + auto it = original_names.find(col.name); + if (it != original_names.end()) + { + String original = it->second; + src_names.push_back(original); + dst_columns.push_back({col.name, col.type}); + } + else + { + /// Can't extract column from dictionary table + /// TODO: Sometimes it should be possible to reconstruct required column, + /// e.g. if it's an expression depending on dictionary attributes + return false; + } + } + dictionary_reader = std::make_shared(dict_name, src_names, dst_columns, context); + + return true; +} + static void renameIfNeeded(String & name, const NameToNameMap & renames) { if (const auto it = renames.find(name); it != renames.end()) @@ -473,7 +558,7 @@ TableJoin::createConvertingActions( log_actions("Right", right_converting_actions); } - forAllKeys(clauses, [&](auto & left_key, auto & right_key) + forKeyNamesInJoinClauses(clauses, [&](auto & left_key, auto & right_key) { renameIfNeeded(left_key, left_key_column_rename); renameIfNeeded(right_key, right_key_column_rename); @@ -507,7 +592,7 @@ void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig throw DB::Exception("ASOF join over right table Nullable column is not implemented", ErrorCodes::NOT_IMPLEMENTED); } - forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) + forKeyNamesInJoinClauses(clauses, [&](const auto & left_key_name, const auto & right_key_name) { auto ltypeit = left_types.find(left_key_name); auto rtypeit = right_types.find(right_key_name); @@ -708,7 +793,7 @@ std::unordered_map TableJoin::leftToRightKeyRemap() const if (hasUsing()) { const auto & required_right_keys = requiredRightKeys(); - forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) + forKeyNamesInJoinClauses(clauses, [&](const auto & left_key_name, const auto & right_key_name) { if (!required_right_keys.contains(right_key_name)) left_to_right_key_remap[left_key_name] = right_key_name; @@ -722,14 +807,16 @@ Names TableJoin::getAllNames(JoinTableSide side) const { Names res; auto func = [&res](const auto & name) { res.emplace_back(name); return true; }; + if (side == JoinTableSide::Left) - forAllKeys(clauses, func); + forKeyNamesInJoinClauses(clauses, func); else - forAllKeys(clauses, func); + forKeyNamesInJoinClauses(clauses, func); + return res; } -void TableJoin::assertHasOneOnExpr() const +void TableJoin::assertHasSingleClause() const { if (!oneDisjunct()) { diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 3835ef77deb..cfa05cb7de1 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -55,15 +55,26 @@ public: ASTPtr on_filter_condition_left; ASTPtr on_filter_condition_right; + std::string analyzer_left_filter_condition_column_name; + std::string analyzer_right_filter_condition_column_name; + JoinOnClause() = default; std::pair condColumnNames() const { std::pair res; + + if (!analyzer_left_filter_condition_column_name.empty()) + res.first = analyzer_left_filter_condition_column_name; + + if (!analyzer_right_filter_condition_column_name.empty()) + res.second = analyzer_right_filter_condition_column_name; + if (on_filter_condition_left) res.first = on_filter_condition_left->getColumnName(); if (on_filter_condition_right) res.second = on_filter_condition_right->getColumnName(); + return res; } @@ -111,9 +122,6 @@ private: * to the subquery will be added expression `expr(t2 columns)`. * It's possible to use name `expr(t2 columns)`. */ - - friend class TreeRewriter; - SizeLimits size_limits; const size_t default_max_bytes = 0; const bool join_use_nulls = false; @@ -124,9 +132,6 @@ private: const size_t max_files_to_merge = 0; const String temporary_files_codec = "LZ4"; - /// the limit has no technical reasons, it supposed to improve safety - const size_t MAX_DISJUNCTS = 16; /// NOLINT - ASTs key_asts_left; ASTs key_asts_right; @@ -170,7 +175,7 @@ private: void addKey(const String & left_name, const String & right_name, const ASTPtr & left_ast, const ASTPtr & right_ast = nullptr); - void assertHasOneOnExpr() const; + void assertHasSingleClause() const; /// Calculates common supertypes for corresponding join key columns. template @@ -178,6 +183,8 @@ private: NamesAndTypesList correctedColumnsAddedByJoin() const; + void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix); + public: TableJoin() = default; @@ -217,8 +224,8 @@ public: bool allowParallelHashJoin() const; bool joinUseNulls() const { return join_use_nulls; } - bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(table_join.kind); } - bool forceNullableLeft() const { return join_use_nulls && isRightOrFull(table_join.kind); } + bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(kind()); } + bool forceNullableLeft() const { return join_use_nulls && isRightOrFull(kind()); } size_t defaultMaxBytes() const { return default_max_bytes; } size_t maxJoinedBlockRows() const { return max_joined_block_rows; } size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; } @@ -229,8 +236,11 @@ public: bool oneDisjunct() const; - JoinOnClause & getOnlyClause() { assertHasOneOnExpr(); return clauses[0]; } - const JoinOnClause & getOnlyClause() const { assertHasOneOnExpr(); return clauses[0]; } + ASTTableJoin & getTableJoin() { return table_join; } + const ASTTableJoin & getTableJoin() const { return table_join; } + + JoinOnClause & getOnlyClause() { assertHasSingleClause(); return clauses[0]; } + const JoinOnClause & getOnlyClause() const { assertHasSingleClause(); return clauses[0]; } std::vector & getClauses() { return clauses; } const std::vector & getClauses() const { return clauses; } @@ -266,13 +276,13 @@ public: NamesWithAliases getNamesWithAliases(const NameSet & required_columns) const; NamesWithAliases getRequiredColumns(const Block & sample, const Names & action_required_columns) const; - void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix); size_t rightKeyInclusion(const String & name) const; NameSet requiredRightKeys() const; bool leftBecomeNullable(const DataTypePtr & column_type) const; bool rightBecomeNullable(const DataTypePtr & column_type) const; void addJoinedColumn(const NameAndTypePair & joined_column); + void setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value); template void addJoinedColumnsAndCorrectTypesImpl(TColumns & left_columns, bool correct_nullability); @@ -294,15 +304,9 @@ public: ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only + void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix); const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } - - Names columnsAddedByJoin() const - { - Names res; - for (const auto & col : columns_added_by_join) - res.push_back(col.name); - return res; - } + const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; } /// StorageJoin overrides key names (cause of different names qualification) void setRightKeys(const Names & keys) { getOnlyClause().key_names_right = keys; } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index ac49d79c6ba..da12dccd8d8 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -611,7 +611,7 @@ void getArrayJoinedColumns(ASTPtr & query, TreeRewriterResult & result, const AS } } -void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, bool old_any, ASTTableJoin & out_table_join) +void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, bool old_any, std::shared_ptr & analyzed_join) { const ASTTablesInSelectQueryElement * node = select_query.join(); if (!node) @@ -649,7 +649,7 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul throw Exception("ANY FULL JOINs are not implemented", ErrorCodes::NOT_IMPLEMENTED); } - out_table_join = table_join; + analyzed_join->getTableJoin() = table_join; } /// Evaluate expression and return boolean value if it can be interpreted as bool. @@ -1236,14 +1236,11 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( if (tables_with_columns.size() > 1) { const auto & right_table = tables_with_columns[1]; - auto & cols_from_joined = result.analyzed_join->columns_from_joined_table; - cols_from_joined = right_table.columns; + auto columns_from_joined_table = right_table.columns; /// query can use materialized or aliased columns from right joined table, /// we want to request it for right table - cols_from_joined.insert(cols_from_joined.end(), right_table.hidden_columns.begin(), right_table.hidden_columns.end()); - - result.analyzed_join->deduplicateAndQualifyColumnNames( - source_columns_set, right_table.table.getQualifiedNamePrefix()); + columns_from_joined_table.insert(columns_from_joined_table.end(), right_table.hidden_columns.begin(), right_table.hidden_columns.end()); + result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix()); } translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns); @@ -1254,7 +1251,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( NameSet all_source_columns_set = source_columns_set; if (table_join) { - for (const auto & [name, _] : table_join->columns_from_joined_table) + for (const auto & [name, _] : table_join->columnsFromJoinedTable()) all_source_columns_set.insert(name); } @@ -1304,7 +1301,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set); setJoinStrictness( - *select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, result.analyzed_join->table_join); + *select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, result.analyzed_join); auto * table_join_ast = select_query->join() ? select_query->join()->table_join->as() : nullptr; if (table_join_ast && tables_with_columns.size() >= 2) diff --git a/tests/queries/0_stateless/02337_analyzer_columns_basic.sql b/tests/queries/0_stateless/02337_analyzer_columns_basic.sql index 0f9efdb95cd..e03a65a4e4f 100644 --- a/tests/queries/0_stateless/02337_analyzer_columns_basic.sql +++ b/tests/queries/0_stateless/02337_analyzer_columns_basic.sql @@ -29,6 +29,7 @@ INSERT INTO test_table VALUES (0, 'Value'); SELECT 'Table access without table name qualification'; SELECT test_id FROM test_table; -- { serverError 47 } +SELECT test_id FROM test_unknown_table; -- { serverError 60 } DESCRIBE (SELECT id FROM test_table); SELECT id FROM test_table; diff --git a/tests/queries/0_stateless/02369_analyzer_array_join_function.reference b/tests/queries/0_stateless/02369_analyzer_array_join_function.reference index 8fa16a27152..7025cff6909 100644 --- a/tests/queries/0_stateless/02369_analyzer_array_join_function.reference +++ b/tests/queries/0_stateless/02369_analyzer_array_join_function.reference @@ -42,41 +42,3 @@ 0 1 1 0 2 2 0 3 3 --- -0 1 1 -0 1 2 -0 1 3 -0 1 4 -0 2 1 -0 2 2 -0 2 3 -0 2 4 -0 3 1 -0 3 2 -0 3 3 -0 3 4 --- -0 1 1 5 -0 1 1 6 -0 1 2 5 -0 1 2 6 -0 1 3 5 -0 1 3 6 -0 1 4 5 -0 1 4 6 -0 2 1 5 -0 2 1 6 -0 2 2 5 -0 2 2 6 -0 2 3 5 -0 2 3 6 -0 2 4 5 -0 2 4 6 -0 3 1 5 -0 3 1 6 -0 3 2 5 -0 3 2 6 -0 3 3 5 -0 3 3 6 -0 3 4 5 -0 3 4 6 diff --git a/tests/queries/0_stateless/02369_analyzer_array_join_function.sql b/tests/queries/0_stateless/02369_analyzer_array_join_function.sql index 08b3bc3c146..db9884f9cdd 100644 --- a/tests/queries/0_stateless/02369_analyzer_array_join_function.sql +++ b/tests/queries/0_stateless/02369_analyzer_array_join_function.sql @@ -1,6 +1,5 @@ SET use_analyzer = 1; - SELECT arrayJoin([1, 2, 3]); SELECT '--'; @@ -49,12 +48,12 @@ SELECT '--'; SELECT id, arrayJoin(value_1) AS a, a FROM test_table; -SELECT '--'; +-- SELECT '--'; -SELECT id, arrayJoin(value_1), arrayJoin(value_2) FROM test_table; +-- SELECT id, arrayJoin(value_1), arrayJoin(value_2) FROM test_table; -SELECT '--'; +-- SELECT '--'; -SELECT id, arrayJoin(value_1), arrayJoin(value_2), arrayJoin([5, 6]) FROM test_table; +-- SELECT id, arrayJoin(value_1), arrayJoin(value_2), arrayJoin([5, 6]) FROM test_table; DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02371_analyzer_join_cross.reference b/tests/queries/0_stateless/02371_analyzer_join_cross.reference new file mode 100644 index 00000000000..50e43ac28d1 --- /dev/null +++ b/tests/queries/0_stateless/02371_analyzer_join_cross.reference @@ -0,0 +1,133 @@ +0 Join_1_Value_0 0 Join_2_Value_0 +0 Join_1_Value_0 1 Join_2_Value_1 +0 Join_1_Value_0 2 Join_2_Value_2 +1 Join_1_Value_1 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +1 Join_1_Value_1 2 Join_2_Value_2 +3 Join_1_Value_3 0 Join_2_Value_0 +3 Join_1_Value_3 1 Join_2_Value_1 +3 Join_1_Value_3 2 Join_2_Value_2 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +0 Join_1_Value_0 1 Join_2_Value_1 +0 Join_1_Value_0 2 Join_2_Value_2 +1 Join_1_Value_1 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +1 Join_1_Value_1 2 Join_2_Value_2 +3 Join_1_Value_3 0 Join_2_Value_0 +3 Join_1_Value_3 1 Join_2_Value_1 +3 Join_1_Value_3 2 Join_2_Value_2 +-- +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 +0 0 Join_1_Value_0 Join_1_Value_0 1 1 Join_2_Value_1 Join_2_Value_1 +0 0 Join_1_Value_0 Join_1_Value_0 2 2 Join_2_Value_2 Join_2_Value_2 +1 1 Join_1_Value_1 Join_1_Value_1 0 0 Join_2_Value_0 Join_2_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 +1 1 Join_1_Value_1 Join_1_Value_1 2 2 Join_2_Value_2 Join_2_Value_2 +3 3 Join_1_Value_3 Join_1_Value_3 0 0 Join_2_Value_0 Join_2_Value_0 +3 3 Join_1_Value_3 Join_1_Value_3 1 1 Join_2_Value_1 Join_2_Value_1 +3 3 Join_1_Value_3 Join_1_Value_3 2 2 Join_2_Value_2 Join_2_Value_2 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +0 Join_1_Value_0 1 Join_2_Value_1 +0 Join_1_Value_0 2 Join_2_Value_2 +1 Join_1_Value_1 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +1 Join_1_Value_1 2 Join_2_Value_2 +3 Join_1_Value_3 0 Join_2_Value_0 +3 Join_1_Value_3 1 Join_2_Value_1 +3 Join_1_Value_3 2 Join_2_Value_2 +-- +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 +0 0 Join_1_Value_0 Join_1_Value_0 1 1 Join_2_Value_1 Join_2_Value_1 +0 0 Join_1_Value_0 Join_1_Value_0 2 2 Join_2_Value_2 Join_2_Value_2 +1 1 Join_1_Value_1 Join_1_Value_1 0 0 Join_2_Value_0 Join_2_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 +1 1 Join_1_Value_1 Join_1_Value_1 2 2 Join_2_Value_2 Join_2_Value_2 +3 3 Join_1_Value_3 Join_1_Value_3 0 0 Join_2_Value_0 Join_2_Value_0 +3 3 Join_1_Value_3 Join_1_Value_3 1 1 Join_2_Value_1 Join_2_Value_1 +3 3 Join_1_Value_3 Join_1_Value_3 2 2 Join_2_Value_2 Join_2_Value_2 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 Join_1_Value_0 0 Join_2_Value_0 1 Join_3_Value_1 +0 Join_1_Value_0 0 Join_2_Value_0 2 Join_3_Value_2 +0 Join_1_Value_0 1 Join_2_Value_1 0 Join_3_Value_0 +0 Join_1_Value_0 1 Join_2_Value_1 1 Join_3_Value_1 +0 Join_1_Value_0 1 Join_2_Value_1 2 Join_3_Value_2 +0 Join_1_Value_0 2 Join_2_Value_2 0 Join_3_Value_0 +0 Join_1_Value_0 2 Join_2_Value_2 1 Join_3_Value_1 +0 Join_1_Value_0 2 Join_2_Value_2 2 Join_3_Value_2 +1 Join_1_Value_1 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 Join_2_Value_0 1 Join_3_Value_1 +1 Join_1_Value_1 0 Join_2_Value_0 2 Join_3_Value_2 +1 Join_1_Value_1 1 Join_2_Value_1 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +1 Join_1_Value_1 1 Join_2_Value_1 2 Join_3_Value_2 +1 Join_1_Value_1 2 Join_2_Value_2 0 Join_3_Value_0 +1 Join_1_Value_1 2 Join_2_Value_2 1 Join_3_Value_1 +1 Join_1_Value_1 2 Join_2_Value_2 2 Join_3_Value_2 +3 Join_1_Value_3 0 Join_2_Value_0 0 Join_3_Value_0 +3 Join_1_Value_3 0 Join_2_Value_0 1 Join_3_Value_1 +3 Join_1_Value_3 0 Join_2_Value_0 2 Join_3_Value_2 +3 Join_1_Value_3 1 Join_2_Value_1 0 Join_3_Value_0 +3 Join_1_Value_3 1 Join_2_Value_1 1 Join_3_Value_1 +3 Join_1_Value_3 1 Join_2_Value_1 2 Join_3_Value_2 +3 Join_1_Value_3 2 Join_2_Value_2 0 Join_3_Value_0 +3 Join_1_Value_3 2 Join_2_Value_2 1 Join_3_Value_1 +3 Join_1_Value_3 2 Join_2_Value_2 2 Join_3_Value_2 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 Join_1_Value_0 0 Join_2_Value_0 1 Join_3_Value_1 +0 Join_1_Value_0 0 Join_2_Value_0 2 Join_3_Value_2 +0 Join_1_Value_0 1 Join_2_Value_1 0 Join_3_Value_0 +0 Join_1_Value_0 1 Join_2_Value_1 1 Join_3_Value_1 +0 Join_1_Value_0 1 Join_2_Value_1 2 Join_3_Value_2 +0 Join_1_Value_0 2 Join_2_Value_2 0 Join_3_Value_0 +0 Join_1_Value_0 2 Join_2_Value_2 1 Join_3_Value_1 +0 Join_1_Value_0 2 Join_2_Value_2 2 Join_3_Value_2 +1 Join_1_Value_1 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 Join_2_Value_0 1 Join_3_Value_1 +1 Join_1_Value_1 0 Join_2_Value_0 2 Join_3_Value_2 +1 Join_1_Value_1 1 Join_2_Value_1 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +1 Join_1_Value_1 1 Join_2_Value_1 2 Join_3_Value_2 +1 Join_1_Value_1 2 Join_2_Value_2 0 Join_3_Value_0 +1 Join_1_Value_1 2 Join_2_Value_2 1 Join_3_Value_1 +1 Join_1_Value_1 2 Join_2_Value_2 2 Join_3_Value_2 +3 Join_1_Value_3 0 Join_2_Value_0 0 Join_3_Value_0 +3 Join_1_Value_3 0 Join_2_Value_0 1 Join_3_Value_1 +3 Join_1_Value_3 0 Join_2_Value_0 2 Join_3_Value_2 +3 Join_1_Value_3 1 Join_2_Value_1 0 Join_3_Value_0 +3 Join_1_Value_3 1 Join_2_Value_1 1 Join_3_Value_1 +3 Join_1_Value_3 1 Join_2_Value_1 2 Join_3_Value_2 +3 Join_1_Value_3 2 Join_2_Value_2 0 Join_3_Value_0 +3 Join_1_Value_3 2 Join_2_Value_2 1 Join_3_Value_1 +3 Join_1_Value_3 2 Join_2_Value_2 2 Join_3_Value_2 +-- +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_3_Value_1 Join_3_Value_1 +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 2 2 Join_3_Value_2 Join_3_Value_2 +0 0 Join_1_Value_0 Join_1_Value_0 1 1 Join_2_Value_1 Join_2_Value_1 0 0 Join_3_Value_0 Join_3_Value_0 +0 0 Join_1_Value_0 Join_1_Value_0 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +0 0 Join_1_Value_0 Join_1_Value_0 1 1 Join_2_Value_1 Join_2_Value_1 2 2 Join_3_Value_2 Join_3_Value_2 +0 0 Join_1_Value_0 Join_1_Value_0 2 2 Join_2_Value_2 Join_2_Value_2 0 0 Join_3_Value_0 Join_3_Value_0 +0 0 Join_1_Value_0 Join_1_Value_0 2 2 Join_2_Value_2 Join_2_Value_2 1 1 Join_3_Value_1 Join_3_Value_1 +0 0 Join_1_Value_0 Join_1_Value_0 2 2 Join_2_Value_2 Join_2_Value_2 2 2 Join_3_Value_2 Join_3_Value_2 +1 1 Join_1_Value_1 Join_1_Value_1 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_3_Value_1 Join_3_Value_1 +1 1 Join_1_Value_1 Join_1_Value_1 0 0 Join_2_Value_0 Join_2_Value_0 2 2 Join_3_Value_2 Join_3_Value_2 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 2 2 Join_3_Value_2 Join_3_Value_2 +1 1 Join_1_Value_1 Join_1_Value_1 2 2 Join_2_Value_2 Join_2_Value_2 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 2 2 Join_2_Value_2 Join_2_Value_2 1 1 Join_3_Value_1 Join_3_Value_1 +1 1 Join_1_Value_1 Join_1_Value_1 2 2 Join_2_Value_2 Join_2_Value_2 2 2 Join_3_Value_2 Join_3_Value_2 +3 3 Join_1_Value_3 Join_1_Value_3 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +3 3 Join_1_Value_3 Join_1_Value_3 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_3_Value_1 Join_3_Value_1 +3 3 Join_1_Value_3 Join_1_Value_3 0 0 Join_2_Value_0 Join_2_Value_0 2 2 Join_3_Value_2 Join_3_Value_2 +3 3 Join_1_Value_3 Join_1_Value_3 1 1 Join_2_Value_1 Join_2_Value_1 0 0 Join_3_Value_0 Join_3_Value_0 +3 3 Join_1_Value_3 Join_1_Value_3 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +3 3 Join_1_Value_3 Join_1_Value_3 1 1 Join_2_Value_1 Join_2_Value_1 2 2 Join_3_Value_2 Join_3_Value_2 +3 3 Join_1_Value_3 Join_1_Value_3 2 2 Join_2_Value_2 Join_2_Value_2 0 0 Join_3_Value_0 Join_3_Value_0 +3 3 Join_1_Value_3 Join_1_Value_3 2 2 Join_2_Value_2 Join_2_Value_2 1 1 Join_3_Value_1 Join_3_Value_1 +3 3 Join_1_Value_3 Join_1_Value_3 2 2 Join_2_Value_2 Join_2_Value_2 2 2 Join_3_Value_2 Join_3_Value_2 diff --git a/tests/queries/0_stateless/02371_analyzer_join_cross.sql b/tests/queries/0_stateless/02371_analyzer_join_cross.sql new file mode 100644 index 00000000000..b7d29049398 --- /dev/null +++ b/tests/queries/0_stateless/02371_analyzer_join_cross.sql @@ -0,0 +1,78 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table_join_1; +CREATE TABLE test_table_join_1 +( + id UInt64, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_2; +CREATE TABLE test_table_join_2 +( + id UInt64, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_3; +CREATE TABLE test_table_join_3 +( + id UInt64, + value String +) ENGINE = TinyLog; + +INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value_0'); +INSERT INTO test_table_join_1 VALUES (1, 'Join_1_Value_1'); +INSERT INTO test_table_join_1 VALUES (3, 'Join_1_Value_3'); + +INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value_0'); +INSERT INTO test_table_join_2 VALUES (1, 'Join_2_Value_1'); +INSERT INTO test_table_join_2 VALUES (2, 'Join_2_Value_2'); + +INSERT INTO test_table_join_3 VALUES (0, 'Join_3_Value_0'); +INSERT INTO test_table_join_3 VALUES (1, 'Join_3_Value_1'); +INSERT INTO test_table_join_3 VALUES (2, 'Join_3_Value_2'); + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value +FROM test_table_join_1, test_table_join_2; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1, test_table_join_2 AS t2; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value +FROM test_table_join_1 AS t1, test_table_join_2 AS t2; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1, test_table_join_2 AS t2; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value FROM test_table_join_1 AS t1, test_table_join_2 AS t2; + +SELECT '--'; + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1, test_table_join_2, test_table_join_3; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1, test_table_join_2 AS t2, test_table_join_3 AS t3; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1, test_table_join_2 AS t2, test_table_join_3 AS t3; + +SELECT id FROM test_table_join_1, test_table_join_2; -- { serverError 36 } + +SELECT value FROM test_table_join_1, test_table_join_2; -- { serverError 36 } + +DROP TABLE test_table_join_1; +DROP TABLE test_table_join_2; +DROP TABLE test_table_join_3; diff --git a/tests/queries/0_stateless/02372_analyzer_join_inner.reference b/tests/queries/0_stateless/02372_analyzer_join_inner.reference new file mode 100644 index 00000000000..3f8435c35d4 --- /dev/null +++ b/tests/queries/0_stateless/02372_analyzer_join_inner.reference @@ -0,0 +1,51 @@ +Join without ON conditions +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +-- +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +-- +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +Join with ON conditions +0 Join_1_Value_0 0 Join_2_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +Join only join expression use keys +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +-- +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +-- +Join_1_Value_0 Join_2_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_2_Value_1 +-- +Join_1_Value_0 Join_2_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_2_Value_1 diff --git a/tests/queries/0_stateless/02372_analyzer_join_inner.sql b/tests/queries/0_stateless/02372_analyzer_join_inner.sql new file mode 100644 index 00000000000..5c915deef56 --- /dev/null +++ b/tests/queries/0_stateless/02372_analyzer_join_inner.sql @@ -0,0 +1,156 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table_join_1; +CREATE TABLE test_table_join_1 +( + id UInt64, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_2; +CREATE TABLE test_table_join_2 +( + id UInt64, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_3; +CREATE TABLE test_table_join_3 +( + id UInt64, + value String +) ENGINE = TinyLog; + +INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value_0'); +INSERT INTO test_table_join_1 VALUES (1, 'Join_1_Value_1'); +INSERT INTO test_table_join_1 VALUES (3, 'Join_1_Value_3'); + +INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value_0'); +INSERT INTO test_table_join_2 VALUES (1, 'Join_2_Value_1'); +INSERT INTO test_table_join_2 VALUES (2, 'Join_2_Value_2'); + +INSERT INTO test_table_join_3 VALUES (0, 'Join_3_Value_0'); +INSERT INTO test_table_join_3 VALUES (1, 'Join_3_Value_1'); +INSERT INTO test_table_join_3 VALUES (2, 'Join_3_Value_2'); + +SELECT 'Join without ON conditions'; + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; + +SELECT '--'; + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id + INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id + INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id + INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id; + +SELECT id FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } + +SELECT value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } + +SELECT 'Join with ON conditions'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' + INNER JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; + +SELECT 'Join only join expression use keys'; + +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id +INNER JOIN test_table_join_2 AS t3 ON t2.id = t3.id; + +SELECT '--'; + +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id +INNER JOIN test_table_join_2 AS t3 ON t1.id = t3.id; + + +DROP TABLE test_table_join_1; +DROP TABLE test_table_join_2; +DROP TABLE test_table_join_3; From b32092f24ac3a94cf2355a68af4452a571655d97 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 20 Aug 2022 16:01:50 +0200 Subject: [PATCH 067/188] JOIN support join_use_nulls, USING --- src/Analyzer/ColumnNode.cpp | 33 +- src/Analyzer/ColumnNode.h | 48 +- src/Analyzer/QueryAnalysisPass.cpp | 299 +++++-- .../InterpreterSelectQueryAnalyzer.cpp | 817 +++++++++++------- .../0_stateless/02372_analyzer_join.reference | 740 ++++++++++++++++ .../0_stateless/02372_analyzer_join.sql.j2 | 180 ++++ .../02372_analyzer_join_inner.reference | 51 -- .../0_stateless/02372_analyzer_join_inner.sql | 156 ---- 8 files changed, 1681 insertions(+), 643 deletions(-) create mode 100644 tests/queries/0_stateless/02372_analyzer_join.reference create mode 100644 tests/queries/0_stateless/02372_analyzer_join.sql.j2 delete mode 100644 tests/queries/0_stateless/02372_analyzer_join_inner.reference delete mode 100644 tests/queries/0_stateless/02372_analyzer_join_inner.sql diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index 5fdce4c574c..07cb4e377d8 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -31,22 +31,6 @@ ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodePtr expression_node children[expression_child_index] = std::move(expression_node_); } -ColumnNode::ColumnNode(NameAndTypePair column_, String column_name_qualification_, QueryTreeNodeWeakPtr column_source_) - : column(std::move(column_)) - , column_name_qualification(std::move(column_name_qualification_)) - , column_source(std::move(column_source_)) -{ -} - -ColumnNode::ColumnNode(NameAndTypePair column_, String column_name_qualification_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_) - : column(std::move(column_)) - , column_name_qualification(std::move(column_name_qualification_)) - , column_source(std::move(column_source_)) -{ - children.resize(children_size); - children[expression_child_index] = std::move(expression_node_); -} - QueryTreeNodePtr ColumnNode::getColumnSource() const { auto lock = column_source.lock(); @@ -68,9 +52,6 @@ void ColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t buffer << ", column_name: " << column.name << ", result_type: " << column.type->getName(); - if (!column_name_qualification.empty()) - buffer << ", column_name_qualification: " << column_name_qualification; - auto column_source_ptr = column_source.lock(); if (column_source_ptr) buffer << ", source_id: " << state.getNodeId(column_source_ptr.get()); @@ -87,7 +68,7 @@ void ColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t bool ColumnNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); - if (column != rhs_typed.column || column_name_qualification != rhs_typed.column_name_qualification) + if (column != rhs_typed.column) return false; auto source_ptr = column_source.lock(); @@ -112,9 +93,6 @@ void ColumnNode::updateTreeHashImpl(HashState & hash_state) const hash_state.update(column_type_name.size()); hash_state.update(column_type_name); - hash_state.update(column_name_qualification.size()); - hash_state.update(column_name_qualification); - auto column_source_ptr = column_source.lock(); if (column_source_ptr) column_source_ptr->updateTreeHashImpl(hash_state); @@ -122,7 +100,10 @@ void ColumnNode::updateTreeHashImpl(HashState & hash_state) const QueryTreeNodePtr ColumnNode::cloneImpl() const { - return std::make_shared(column, column_name_qualification, column_source); + auto clone_result = std::make_shared(column, column_source); + clone_result->display_identifier = display_identifier; + + return clone_result; } void ColumnNode::getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update) @@ -137,9 +118,9 @@ void ColumnNode::getPointersToUpdateAfterClone(QueryTreePointersToUpdate & point ASTPtr ColumnNode::toASTImpl() const { - if (!column_name_qualification.empty()) + if (!display_identifier.empty()) { - std::vector parts = {column_name_qualification, column.name}; + std::vector parts = display_identifier.getParts(); return std::make_shared(std::move(parts)); } diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index 4784064d0d6..2d0894628d4 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -15,14 +15,14 @@ namespace DB * For table ALIAS columns. Column node must contain expression. * For ARRAY JOIN join expression column. Column node must contain expression. * - * Additionaly column must be initialized with column name qualification if there are multiple - * unqualified columns with same name in query scope. - * Example: SELECT a.id, b.id FROM test_table_join_1 AS a, test_table_join_1. - * Both columns a.id and b.id have same unqualified name id. And additionally must be initialized - * with qualification a and b. - * * During query analysis pass identifier node is resolved into column. See IdentifierNode.h. * + * It is also important for client provide display identifier for AST conversion in case of JOINS are used. + * Example: SELECT t1.id, t2.id FROM test_table_1 AS t1, test_table_2 AS t2. + * In this example t1.id will be column with name id, but during conversion to AST, to keep AST valid it is important + * to save column name qualification. Display identifier does not take part in implementation of tree hash, + * or tree equals operator. It is only used during AST conversion. + * * Examples: * SELECT id FROM test_table. id is identifier that must be resolved to column node during query analysis pass. * SELECT lambda(x -> x + 1, [1,2,3]). x is identifier inside lambda that must be resolved to column node during query analysis pass. @@ -44,12 +44,6 @@ public: /// Construct expression column node with column name, type, column expression and column source weak pointer. ColumnNode(NameAndTypePair column_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_); - /// Construct column node with column name, type, column name qualification and column source weak pointer. - ColumnNode(NameAndTypePair column_, String column_name_qualification_, QueryTreeNodeWeakPtr column_source_); - - /// Construct expression column node with column name, type, column name qualification column expression and column source weak pointer. - ColumnNode(NameAndTypePair column_, String column_name_qualification_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_); - /// Get column const NameAndTypePair & getColumn() const { @@ -62,14 +56,15 @@ public: return column.name; } - bool hasColumnNameQualfication() const + /// Set display identifier that will be used during convertion to AST + void setDisplayIdentifier(const Identifier & display_identifier_value) { - return !column_name_qualification.empty(); + display_identifier = display_identifier_value; } - const String & getColumnQualification() const + const Identifier & getDisplayIdentifier() const { - return column_name_qualification; + return display_identifier; } /// Get column type @@ -78,6 +73,12 @@ public: return column.type; } + /// Set column type + void setColumnType(DataTypePtr column_type) + { + column.type = std::move(column_type); + } + bool hasExpression() const { return children[expression_child_index] != nullptr; @@ -93,6 +94,19 @@ public: return children[expression_child_index]; } + QueryTreeNodePtr & getExpressionOrThrow() + { + if (!children[expression_child_index]) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Column expression is not initialized"); + + return children[expression_child_index]; + } + + void setExpression(QueryTreeNodePtr expression_value) + { + children[expression_child_index] = std::move(expression_value); + } + /** Get column source. * If column source is not valid logical exception is thrown. */ @@ -134,8 +148,8 @@ protected: private: NameAndTypePair column; - String column_name_qualification; QueryTreeNodeWeakPtr column_source; + Identifier display_identifier; static constexpr size_t expression_child_index = 0; static constexpr size_t children_size = expression_child_index + 1; diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index b31457f57c0..7b6fd4b3bf4 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -165,7 +166,6 @@ namespace ErrorCodes * lookup should not be continued, and exception must be throwed because if lookup continues identifier can be resolved from parent scope. * * TODO: Add expression name into query tree node. Example: SELECT plus(1, 1). Result: SELECT 2. Expression name of constant node should be 2. - * TODO: Disable identifier cache during join tree resolve * TODO: Update exception messages * TODO: JOIN TREE subquery constant columns * TODO: Column name qualifications @@ -174,7 +174,7 @@ namespace ErrorCodes * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. * TODO: UNION - * TODO: JOIN support LEFT, RIGHT with nullable. JOIN support USING. JOIN support SELF JOIN with MergeTree. JOIN support matchers. + * TODO: JOIN support SELF JOIN with MergeTree. JOIN support matchers. * TODO: WINDOW functions * TODO: Table expression modifiers final, sample_size, sample_offset */ @@ -190,7 +190,7 @@ enum class IdentifierLookupContext : uint8_t TABLE, }; -static const char * toString(IdentifierLookupContext identifier_lookup_context) +const char * toString(IdentifierLookupContext identifier_lookup_context) { switch (identifier_lookup_context) { @@ -200,7 +200,7 @@ static const char * toString(IdentifierLookupContext identifier_lookup_context) } } -static const char * toStringLowercase(IdentifierLookupContext identifier_lookup_context) +const char * toStringLowercase(IdentifierLookupContext identifier_lookup_context) { switch (identifier_lookup_context) { @@ -269,7 +269,7 @@ enum class IdentifierResolvePlace : UInt8 DATABASE_CATALOG }; -static const char * toString(IdentifierResolvePlace resolved_identifier_place) +const char * toString(IdentifierResolvePlace resolved_identifier_place) { switch (resolved_identifier_place) { @@ -438,11 +438,11 @@ public: return expressions.empty(); } - void dump(WriteBuffer & buffer) + void dump(WriteBuffer & buffer) const { buffer << expressions.size() << '\n'; - for (auto & [expression, alias] : expressions) + for (const auto & [expression, alias] : expressions) { buffer << "Expression "; buffer << expression->formatASTForErrorMessage(); @@ -454,6 +454,14 @@ public: } } + [[maybe_unused]] String dump() const + { + WriteBufferFromOwnString buffer; + dump(buffer); + + return buffer.str(); + } + private: std::vector> expressions; std::unordered_map alias_name_to_expressions_size; @@ -505,8 +513,11 @@ struct IdentifierResolveScope /// Allow to check parent scopes if identifier cannot be resolved in current scope bool allow_to_check_parent_scopes = true; + /// Use identifier lookup to result cache + bool use_identifier_lookup_to_result_cache = true; + /// Dump identifier resolve scope - [[maybe_unused]] void dump(WriteBuffer & buffer) + [[maybe_unused]] void dump(WriteBuffer & buffer) const { buffer << "Scope node " << scope_node->formatASTForErrorMessage() << '\n'; buffer << "Identifier lookup to result " << identifier_lookup_to_result.size() << '\n'; @@ -521,23 +532,23 @@ struct IdentifierResolveScope for (const auto & [alias_name, node] : expression_argument_name_to_node) buffer << "Alias name " << alias_name << " node " << node->formatASTForErrorMessage() << '\n'; - buffer << "Alias name to expression node " << alias_name_to_expression_node.size() << '\n'; + buffer << "Alias name to expression node table size " << alias_name_to_expression_node.size() << '\n'; for (const auto & [alias_name, node] : alias_name_to_expression_node) - buffer << "Alias name " << alias_name << " expression node " << node->formatASTForErrorMessage() << '\n'; + buffer << "Alias name " << alias_name << " expression node " << node->dumpTree() << '\n'; - buffer << "Alias name to function node " << alias_name_to_lambda_node.size() << '\n'; + buffer << "Alias name to function node table size " << alias_name_to_lambda_node.size() << '\n'; for (const auto & [alias_name, node] : alias_name_to_lambda_node) buffer << "Alias name " << alias_name << " lambda node " << node->formatASTForErrorMessage() << '\n'; - buffer << "Alias name to table expression node " << alias_name_to_table_expression_node.size() << '\n'; + buffer << "Alias name to table expression node table size " << alias_name_to_table_expression_node.size() << '\n'; for (const auto & [alias_name, node] : alias_name_to_table_expression_node) buffer << "Alias name " << alias_name << " node " << node->formatASTForErrorMessage() << '\n'; - buffer << "CTE name to query node " << cte_name_to_query_node.size() << '\n'; + buffer << "CTE name to query node table size " << cte_name_to_query_node.size() << '\n'; for (const auto & [cte_name, node] : cte_name_to_query_node) buffer << "CTE name " << cte_name << " node " << node->formatASTForErrorMessage() << '\n'; - buffer << "Nodes with duplicated aliases " << nodes_with_duplicated_aliases.size() << '\n'; + buffer << "Nodes with duplicated aliases size " << nodes_with_duplicated_aliases.size() << '\n'; for (const auto & node : nodes_with_duplicated_aliases) buffer << "Alias name " << node->getAlias() << " node " << node->formatASTForErrorMessage() << '\n'; @@ -547,6 +558,14 @@ struct IdentifierResolveScope buffer << "Allow to check parent scopes " << allow_to_check_parent_scopes << '\n'; // buffer << "Parent scope " << parent_scope << '\n'; } + + [[maybe_unused]] String dump() const + { + WriteBufferFromOwnString buffer; + dump(buffer); + + return buffer.str(); + } }; @@ -762,6 +781,8 @@ private: QueryTreeNodePtr tryResolveIdentifierFromTable(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + QueryTreeNodePtr tryResolveIdentifierFromJoin(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + QueryTreeNodePtr tryResolveIdentifierFromJoinTreeNode(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope); QueryTreeNodePtr tryResolveIdentifierFromJoinTree(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); @@ -1328,15 +1349,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo scope.scope_node->formatASTForErrorMessage()); if (identifier_column_qualifier_parts > 0) - { - auto identifier_column_qualifier_view = IdentifierView(identifier); - identifier_column_qualifier_view.popLast(identifier.getPartsSize() - identifier_column_qualifier_parts); - - result_column = std::make_shared(result_column->getColumn(), - std::string(identifier_column_qualifier_view.getFullName()), - result_column->getExpression(), - table_expression_node); - } + result_column->setDisplayIdentifier(identifier); QueryTreeNodePtr result_expression = result_column; @@ -1375,6 +1388,110 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo return {}; } +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) +{ + const auto & from_join_node = table_expression_node->as(); + + std::unordered_map join_using_column_name_to_column_node; + + if (from_join_node.isUsingJoinExpression()) + { + auto & join_using_list = from_join_node.getJoinExpression()->as(); + + for (auto & join_using_node : join_using_list.getNodes()) + { + auto & column_node = join_using_node->as(); + join_using_column_name_to_column_node.emplace(column_node.getName(), std::static_pointer_cast(join_using_node)); + } + } + + auto left_resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_join_node.getLeftTableExpression(), scope); + auto right_resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_join_node.getRightTableExpression(), scope); + + std::optional resolved_side; + QueryTreeNodePtr resolved_identifier; + + auto resolve_from_using_column = [&](const QueryTreeNodePtr & using_column, JoinTableSide expression_side, Identifier display_identifier) + { + auto & using_column_node = using_column->as(); + auto & using_expression_list = using_column_node.getExpression()->as(); + + size_t inner_column_node_index = expression_side == JoinTableSide::Left ? 0 : 1; + const auto & inner_column_node = using_expression_list.getNodes().at(inner_column_node_index); + + auto result_column_node = inner_column_node->clone(); + auto & result_column = result_column_node->as(); + result_column.setDisplayIdentifier(display_identifier); + result_column.setColumnType(using_column_node.getColumnType()); + + return result_column_node; + }; + + JoinKind join_kind = from_join_node.getKind(); + + if (left_resolved_identifier && right_resolved_identifier) + { + auto & left_resolved_column = left_resolved_identifier->as(); + auto & right_resolved_column = right_resolved_identifier->as(); + + auto using_column_node_it = join_using_column_name_to_column_node.find(left_resolved_column.getColumnName()); + if (using_column_node_it != join_using_column_name_to_column_node.end() + && left_resolved_column.getColumnName() == right_resolved_column.getColumnName()) + { + JoinTableSide using_column_inner_column_table_side = isRight(join_kind) ? JoinTableSide::Right : JoinTableSide::Left; + resolved_identifier = resolve_from_using_column(using_column_node_it->second, using_column_inner_column_table_side, left_resolved_column.getDisplayIdentifier()); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "JOIN {} ambigious identifier {}. In scope {}", + table_expression_node->formatASTForErrorMessage(), + identifier_lookup.identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + } + } + else if (left_resolved_identifier) + { + resolved_side = JoinTableSide::Left; + auto & left_resolved_column = left_resolved_identifier->as(); + + auto using_column_node_it = join_using_column_name_to_column_node.find(left_resolved_column.getColumnName()); + if (using_column_node_it != join_using_column_name_to_column_node.end()) + resolved_identifier = resolve_from_using_column(using_column_node_it->second, JoinTableSide::Left, left_resolved_column.getDisplayIdentifier()); + else + resolved_identifier = left_resolved_identifier; + } + else if (right_resolved_identifier) + { + resolved_side = JoinTableSide::Right; + auto & right_resolved_column = right_resolved_identifier->as(); + + auto using_column_node_it = join_using_column_name_to_column_node.find(right_resolved_column.getColumnName()); + if (using_column_node_it != join_using_column_name_to_column_node.end()) + resolved_identifier = resolve_from_using_column(using_column_node_it->second, JoinTableSide::Right, right_resolved_column.getDisplayIdentifier()); + else + resolved_identifier = right_resolved_identifier; + } + + bool join_node_in_resolve_process = scope.table_expressions_in_resolve_process.contains(table_expression_node.get()); + if (join_node_in_resolve_process || !resolved_identifier) + return resolved_identifier; + + bool join_use_nulls = context->getSettingsRef().join_use_nulls; + + if (join_use_nulls + && (isFull(join_kind) || + (isLeft(join_kind) && resolved_side && *resolved_side == JoinTableSide::Right) || + (isRight(join_kind) && resolved_side && *resolved_side == JoinTableSide::Left))) + { + resolved_identifier = resolved_identifier->clone(); + auto & resolved_column = resolved_identifier->as(); + resolved_column.setColumnType(makeNullable(resolved_column.getColumnType())); + } + + return resolved_identifier; +} + QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTreeNode(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope) { auto join_tree_node_type = join_tree_node->getNodeType(); @@ -1383,23 +1500,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTreeNode(const Ident { case QueryTreeNodeType::JOIN: { - const auto & from_join_node = join_tree_node->as(); - - auto left_resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_join_node.getLeftTableExpression(), scope); - auto right_resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_join_node.getRightTableExpression(), scope); - - if (left_resolved_identifier && right_resolved_identifier) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "JOIN {} ambigious identifier {}. In scope {}", - join_tree_node->formatASTForErrorMessage(), - identifier_lookup.identifier.getFullName(), - scope.scope_node->formatASTForErrorMessage()); - else if (left_resolved_identifier) - return left_resolved_identifier; - else if (right_resolved_identifier) - return right_resolved_identifier; - - return {}; + return tryResolveIdentifierFromJoin(identifier_lookup, join_tree_node, scope); } case QueryTreeNodeType::ARRAY_JOIN: { @@ -1608,7 +1709,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook identifier_lookup.identifier.getFullName(), scope.scope_node->formatASTForErrorMessage()); - if (!scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup)) + if (scope.use_identifier_lookup_to_result_cache && !scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup)) return it->second; } @@ -1694,7 +1795,9 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook it->second = resolve_result; - if (!resolve_result.resolved_identifier || scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup)) + if (!resolve_result.resolved_identifier || + scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup) || + !scope.use_identifier_lookup_to_result_cache) scope.identifier_lookup_to_result.erase(it); return resolve_result; @@ -2535,7 +2638,9 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes * To support both (SELECT 1) AS expression in projection and (SELECT 1) as subquery in IN, do not use * alias table because in alias table subquery could be evaluated as scalar. */ - bool use_alias_table = !scope.nodes_with_duplicated_aliases.contains(node) || (allow_table_expression && node->getNodeType() == QueryTreeNodeType::QUERY); + bool use_alias_table = true; + if (scope.nodes_with_duplicated_aliases.contains(node) || (allow_table_expression && node->getNodeType() == QueryTreeNodeType::QUERY)) + use_alias_table = false; if (!node_alias.empty() && use_alias_table) { @@ -2749,12 +2854,9 @@ void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, Iden auto result_node_list = std::make_shared(); result_node_list->getNodes().reserve(initial_node_list_size); - for (auto & node : initial_node_list.getNodes()) + auto initial_node_list_nodes_copy = initial_node_list.getNodes(); + for (auto & node : initial_node_list_nodes_copy) { - /// Skip CTE - if (node->as() && node->as()->isCTE()) - continue; - resolveExpressionNode(node, scope, allow_lambda_expression, allow_table_expression); if (auto * expression_list = node->as()) @@ -2854,6 +2956,7 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod { auto & array_join = current_join_tree_node->as(); join_tree_node_ptrs_to_process_queue.push_back(&array_join.getTableExpression()); + scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); break; } case QueryTreeNodeType::JOIN: @@ -2861,6 +2964,7 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod auto & join = current_join_tree_node->as(); join_tree_node_ptrs_to_process_queue.push_back(&join.getLeftTableExpression()); join_tree_node_ptrs_to_process_queue.push_back(&join.getRightTableExpression()); + scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); break; } default: @@ -3131,12 +3235,63 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, resolveQueryJoinTreeNode(join_node.getRightTableExpression(), scope, expressions_visitor); if (join_node.isUsingJoinExpression()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "JOIN {} with USING is unsupported. In scope {}", - join_node.formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); + { + auto & join_using_list = join_node.getJoinExpression()->as(); + for (auto & join_using_node : join_using_list.getNodes()) + { + auto * identifier_node = join_using_node->as(); + if (!identifier_node) + continue; - if (join_node.getJoinExpression()) + const auto & identifier_full_name = identifier_node->getIdentifier().getFullName(); + + IdentifierLookup identifier_lookup {identifier_node->getIdentifier(), IdentifierLookupContext::EXPRESSION}; + auto result_left_table_expression = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_node.getLeftTableExpression(), scope); + if (!result_left_table_expression) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "JOIN {} using identifier {} cannot be resolved from left table expression. In scope {}", + join_node.formatASTForErrorMessage(), + identifier_full_name, + scope.scope_node->formatASTForErrorMessage()); + + auto result_right_table_expression = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_node.getRightTableExpression(), scope); + if (!result_right_table_expression) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "JOIN {} using identifier {} cannot be resolved from right table expression. In scope {}", + join_node.formatASTForErrorMessage(), + identifier_full_name, + scope.scope_node->formatASTForErrorMessage()); + + DataTypePtr common_type; + + try + { + common_type = getLeastSupertype(DataTypes{result_left_table_expression->getResultType(), result_right_table_expression->getResultType()}); + } + catch (Exception & ex) + { + ex.addMessage("JOIN {} cannot infer common type in USING for identifier {}. In scope {}", + join_node.formatASTForErrorMessage(), + identifier_full_name, + scope.scope_node->formatASTForErrorMessage()); + } + + NameAndTypePair join_using_columns_common_name_and_type(identifier_full_name, common_type); + ListNodePtr join_using_expression = std::make_shared(QueryTreeNodes{result_left_table_expression, result_right_table_expression}); + auto join_using_column = std::make_shared(join_using_columns_common_name_and_type, std::move(join_using_expression), join_tree_node); + + join_using_node = std::move(join_using_column); + } + + for (auto & join_using_node : join_using_list.getNodes()) + { + if (!join_using_node->as()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "JOIN {} USING identifier node must be resolved into column node. Actual {}. In scope {}", + join_node.formatASTForErrorMessage(), + join_tree_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + } + else if (join_node.getJoinExpression()) { expressions_visitor.visit(join_node.getJoinExpression()); resolveExpressionNode(join_node.getJoinExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); @@ -3203,9 +3358,11 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.getWhere()) visitor.visit(query_node_typed.getWhere()); - /// Register CTE subqueries + /// Register CTE subqueries and remove them from WITH section - for (auto & node : query_node_typed.getWith().getNodes()) + auto & with_nodes = query_node_typed.getWith().getNodes(); + + for (auto & node : with_nodes) { auto * subquery_node = node->as(); if (!subquery_node || !subquery_node->isCTE()) @@ -3214,13 +3371,28 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier const auto & cte_name = subquery_node->getCTEName(); auto [_, inserted] = data.scope.cte_name_to_query_node.emplace(cte_name, node); if (!inserted) - throw Exception( - ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, + throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, "CTE with name {} already exists. In scope {}", cte_name, data.scope.scope_node->formatASTForErrorMessage()); } + std::erase_if(with_nodes, [](const QueryTreeNodePtr & node) + { + auto * subquery_node = node->as(); + return subquery_node && subquery_node->isCTE(); + }); + + /** Disable identifier cache during JOIN TREE resolve. + * Depending on JOIN expression section, identifier with same name + * can be resolved in different columns. + * + * Example: SELECT id FROM test_table AS t1 INNER JOIN test_table AS t2 ON t1.id = t2.id INNER JOIN test_table AS t3 ON t1.id = t3.id + * In first join expression ON t1.id = t2.id t1.id is resolved into test_table.id column. + * IN second join expression ON t1.id = t3.id t1.id must be resolved into join column that wrap test_table.id column. + */ + scope.use_identifier_lookup_to_result_cache = false; + if (query_node_typed.getFrom()) { TableExpressionsAliasVisitor::Data table_expressions_visitor_data{scope}; @@ -3234,16 +3406,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier resolveQueryJoinTreeNode(query_node_typed.getFrom(), scope, visitor); } - /** Clear cache of identifier lookups. - * It is necessary because during resolve of FROM section we could not use FROM section. - * - * Example: - * WITH a AS (SELECT 1) FROM SELECT a.* FROM a; - * - * During FROM section resolve identifier a is resolved into CTE. - * During qualified matcher a.* resolve identifier a must be resolved into table. - */ - scope.identifier_lookup_to_result.clear(); + scope.use_identifier_lookup_to_result_cache = true; /// Resolve query node sections. @@ -3276,7 +3439,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier { auto node = node_with_duplicated_alias; auto node_alias = node->getAlias(); - resolveExpressionNode(node, scope, true /*allow_lambda_expression*/, true /*allow_table_expression*/); + resolveExpressionNode(node, scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); bool has_node_in_alias_table = false; diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index c2b2d95a94d..848ca04480a 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -30,6 +30,8 @@ #include #include +#include +#include #include #include @@ -62,7 +64,9 @@ namespace ErrorCodes /** ClickHouse query planner. * - * TODO: JOIN support columns cast. JOIN support ASOF. JOIN support strictness. + * TODO: JOIN support ASOF. JOIN support strictness. JOIN support constants + * TODO: JOIN drop unnecessary columns after ON, USING section + * TODO: Support display names * TODO: Support RBAC. Support RBAC for ALIAS columns. * TODO: Support distributed query processing * TODO: Support PREWHERE @@ -75,6 +79,8 @@ namespace ErrorCodes * TODO: Support trivial count optimization * TODO: Support totals, extremes * TODO: Support projections + * TODO: Support read in order optimization + * TODO: Simplify actions chain */ namespace @@ -95,20 +101,6 @@ namespace return query_pipeline_buffer.str(); } -struct TableExpressionColumns -{ - NamesAndTypesList all_columns; - NameSet all_columns_names_set; - - NamesAndTypesList source_input_columns; - NameSet source_columns_set; - std::unordered_map column_name_to_column_identifier; -}; - -using TableExpressionNodeToColumns = std::unordered_map; -using TableExpressionColumnNodeToColumnIdentifier = std::unordered_map; -using ActionsNodeNameToCount = std::unordered_map; - class ActionsChainNode; using ActionsChainNodePtr = std::unique_ptr; using ActionsChainNodes = std::vector; @@ -191,28 +183,6 @@ public: return buffer.str(); } - // NamesAndTypes getAvailableOutputNamesAndTypes() const - // { - // NamesAndTypes result; - // result.reserve(available_output_columns.size()); - - // for (const auto & available_output_column : available_output_columns) - // result.emplace_back(available_output_column.name, available_output_column.type); - - // return result; - // } - - // [[maybe_unused]] Names getAvailableOutputNames() const - // { - // Names result; - // result.reserve(available_output_columns.size()); - - // for (const auto & available_output_column : available_output_columns) - // result.emplace_back(available_output_column.name); - - // return result; - // } - [[maybe_unused]] void addParentIndex(size_t parent_node_index) { parent_nodes_indices.push_back(parent_node_index); @@ -447,80 +417,97 @@ private: ActionsChainNodes nodes; }; -class QueryPlanBuilder +using ColumnIdentifier = std::string; + +struct TableExpressionColumns { -public: - using BuildRootStep = std::function; - using UniteStep = std::function)>; - using BuildStep = std::function; + /// Valid for table, table function, query table expression nodes + NamesAndTypesList source_columns; - explicit QueryPlanBuilder(QueryPlan plan_root_) - { - auto plan_root_ptr = std::make_shared(std::move(plan_root_)); - build_root_step = [plan_root_ptr]() - { - return std::move(*plan_root_ptr); - }; - } + /// Valid for table, table function, query table expression nodes + NameSet source_columns_names; - [[maybe_unused]] explicit QueryPlanBuilder(std::vector plan_builders_, UniteStep unit_step_) - { - auto plan_builders_ptr = std::make_shared>(std::move(plan_builders_)); - build_root_step = [plan_builders_ptr, unite_step = std::move(unit_step_)]() - { - auto plan_builders = std::move(*plan_builders_ptr); - std::vector plans; - plans.reserve(plan_builders.size()); + /// Valid only for table table expression node + NameSet alias_columns; - for (auto && plan_builder : plan_builders) - plans.push_back(std::move(plan_builder).buildPlan()); - - return unite_step(std::move(plans)); - }; - } - - QueryPlanBuilder(QueryPlanBuilder &&) noexcept = default; - [[maybe_unused]] QueryPlanBuilder & operator=(QueryPlanBuilder &&) noexcept = default; - - void addBuildStep(BuildStep step) - { - build_steps.push_back(std::move(step)); - } - - QueryPlan buildPlan() && - { - auto plan = build_root_step(); - - for (auto & build_step : build_steps) - build_step(plan); - - return plan; - } -private: - BuildRootStep build_root_step; - std::vector build_steps; + /// Valid for table, table function, query table expression nodes + std::unordered_map column_name_to_column_identifier; }; +using TableExpressionNodeToColumns = std::unordered_map; + struct PlannerContext { - TableExpressionColumnNodeToColumnIdentifier table_expression_column_node_to_column_identifier; + std::unordered_map column_node_to_column_identifier; + std::unordered_map table_expression_node_to_identifier; + TableExpressionNodeToColumns table_expression_node_to_columns; size_t column_identifier_counter = 0; ActionsChain actions_chain; ActionsDAGPtr where_actions; + size_t where_actions_chain_node_index = 0; std::string where_action_node_name; ActionsDAGPtr projection_actions; ContextPtr query_context; - std::string getColumnUniqueIdentifier() + ColumnIdentifier getColumnUniqueIdentifier(const IQueryTreeNode * column_source_node, std::string column_name = {}) { - auto result = "__column_" + std::to_string(column_identifier_counter); + auto column_unique_prefix = "__column_" + std::to_string(column_identifier_counter); ++column_identifier_counter; - return result; + std::string table_expression_identifier; + auto table_expression_identifier_it = table_expression_node_to_identifier.find(column_source_node); + if (table_expression_identifier_it != table_expression_node_to_identifier.end()) + table_expression_identifier = table_expression_identifier_it->second; + + std::string debug_identifier_suffix; + + if (column_source_node->hasAlias()) + { + debug_identifier_suffix += column_source_node->getAlias(); + } + else if (const auto * table_source_node = column_source_node->as()) + { + debug_identifier_suffix += table_source_node->getStorageID().getFullNameNotQuoted(); + } + else + { + auto column_source_node_type = column_source_node->getNodeType(); + if (column_source_node_type == QueryTreeNodeType::JOIN) + debug_identifier_suffix += "join"; + else if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN) + debug_identifier_suffix += "array_join"; + else if (column_source_node_type == QueryTreeNodeType::TABLE_FUNCTION) + debug_identifier_suffix += "table_function"; + else if (column_source_node_type == QueryTreeNodeType::QUERY) + debug_identifier_suffix += "subquery"; + + if (!table_expression_identifier.empty()) + debug_identifier_suffix += '_' + table_expression_identifier; + } + + if (!column_name.empty()) + debug_identifier_suffix += '.' + column_name; + + if (!debug_identifier_suffix.empty()) + column_unique_prefix += '_' + debug_identifier_suffix; + + return column_unique_prefix; + } + + ColumnIdentifier getColumnIdentifierOrThrow(const IQueryTreeNode * column_source_node) + { + assert(column_source_node->getNodeType() == QueryTreeNodeType::COLUMN); + auto it = column_node_to_column_identifier.find(column_source_node); + if (it == column_node_to_column_identifier.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column identifier is not initialized for column {}", + column_source_node->formatASTForErrorMessage()); + + return it->second; } }; @@ -534,19 +521,19 @@ struct QueryTreeActionsScopeNode node_name_to_node[node.result_name] = &node; } - // bool containsNode(const std::string & node_name) - // { - // return node_name_to_node.find(node_name) != node_name_to_node.end(); - // } + [[maybe_unused]] bool containsNode(const std::string & node_name) + { + return node_name_to_node.find(node_name) != node_name_to_node.end(); + } - // const ActionsDAG::Node * tryGetNode(const std::string & node_name) - // { - // auto it = node_name_to_node.find(node_name); - // if (it == node_name_to_node.end()) - // return {}; + [[maybe_unused]] const ActionsDAG::Node * tryGetNode(const std::string & node_name) + { + auto it = node_name_to_node.find(node_name); + if (it == node_name_to_node.end()) + return {}; - // return it->second; - // } + return it->second; + } const ActionsDAG::Node * getNodeOrThrow(const std::string & node_name) { @@ -889,11 +876,12 @@ private: { case QueryTreeNodeType::COLUMN: { - auto it = planner_context.table_expression_column_node_to_column_identifier.find(node); - if (it == planner_context.table_expression_column_node_to_column_identifier.end()) - return node->getName(); + auto it = planner_context.column_node_to_column_identifier.find(node); + if (it == planner_context.column_node_to_column_identifier.end()) + result = node->getName(); + else + result = it->second; - result = it->second; break; } case QueryTreeNodeType::CONSTANT: @@ -974,6 +962,50 @@ private: const PlannerContext & planner_context; }; +class CollectTableExpressionIdentifiersVisitor +{ +public: + void visit(const QueryTreeNodePtr & join_tree_node, PlannerContext & planner_context) + { + auto join_tree_node_type = join_tree_node->getNodeType(); + + switch (join_tree_node_type) + { + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + std::string table_expression_identifier = std::to_string(planner_context.table_expression_node_to_identifier.size()); + planner_context.table_expression_node_to_identifier.emplace(join_tree_node.get(), table_expression_identifier); + break; + } + case QueryTreeNodeType::JOIN: + { + auto & join_node = join_tree_node->as(); + visit(join_node.getLeftTableExpression(), planner_context); + + std::string table_expression_identifier = std::to_string(planner_context.table_expression_node_to_identifier.size()); + planner_context.table_expression_node_to_identifier.emplace(join_tree_node.get(), table_expression_identifier); + + visit(join_node.getRightTableExpression(), planner_context); + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ARRAY JOIN is not supported"); + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected query, table, table function, join or array join query node. Actual {}", + join_tree_node->formatASTForErrorMessage()); + } + } + } +}; + class CollectSourceColumnsMatcher { public: @@ -994,15 +1026,25 @@ public: auto column_source_node_type = column_source_node->getNodeType(); if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "ARRAY JOIN is not supported"); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ARRAY JOIN is not supported"); if (column_source_node_type == QueryTreeNodeType::LAMBDA) return; + /// JOIN using expression + if (column_node->hasExpression() && column_source_node->getNodeType() == QueryTreeNodeType::JOIN) + return; + + auto & table_expression_node_to_columns = data.planner_context.table_expression_node_to_columns; + auto & table_expression_column_node_to_column_identifier = data.planner_context.column_node_to_column_identifier; + + auto [it, _] = table_expression_node_to_columns.emplace(column_source_node.get(), TableExpressionColumns()); + auto & table_expression_columns = it->second; + if (column_node->hasExpression()) { /// Replace ALIAS column with expression + table_expression_columns.alias_columns.insert(column_node->getColumnName()); node = column_node->getExpression(); visit(node, data); return; @@ -1015,18 +1057,14 @@ public: "Expected table, table function or query column source. Actual {}", column_source_node->formatASTForErrorMessage()); - auto & table_expression_node_to_columns = data.planner_context.table_expression_node_to_columns; - auto & table_expression_column_node_to_column_identifier = data.planner_context.table_expression_column_node_to_column_identifier; - - auto [it, _] = table_expression_node_to_columns.emplace(column_source_node.get(), TableExpressionColumns()); - auto [source_columns_set_it, inserted] = it->second.source_columns_set.insert(column_node->getColumnName()); + auto [source_columns_set_it, inserted] = it->second.source_columns_names.insert(column_node->getColumnName()); if (inserted) { - auto column_identifier = data.planner_context.getColumnUniqueIdentifier(); + auto column_identifier = data.planner_context.getColumnUniqueIdentifier(column_source_node.get(), column_node->getColumnName()); table_expression_column_node_to_column_identifier.emplace(column_node, column_identifier); it->second.column_name_to_column_identifier.emplace(column_node->getColumnName(), column_identifier); - it->second.source_input_columns.emplace_back(column_node->getColumn()); + it->second.source_columns.emplace_back(column_node->getColumn()); } else { @@ -1061,18 +1099,12 @@ ActionsDAGPtr convertExpressionNodeIntoDAG(const QueryTreeNodePtr & expression_n return action_dag; } -struct JoinTreeNodePlan -{ - QueryPlanBuilder plan_builder; - std::vector actions_chain_node_indices; -}; - -JoinTreeNodePlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, +QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, PlannerContext & planner_context); -JoinTreeNodePlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, +QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, SelectQueryInfo & table_expression_query_info, const SelectQueryOptions & select_query_options, PlannerContext & planner_context) @@ -1096,7 +1128,7 @@ JoinTreeNodePlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_express auto from_stage = storage->getQueryProcessingStage(planner_context.query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); - Names column_names(table_expression_columns.source_columns_set.begin(), table_expression_columns.source_columns_set.end()); + Names column_names(table_expression_columns.source_columns_names.begin(), table_expression_columns.source_columns_names.end()); std::optional read_additional_column; @@ -1109,10 +1141,11 @@ JoinTreeNodePlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_express if (read_additional_column) { + auto column_identifier = planner_context.getColumnUniqueIdentifier(table_expression.get(), read_additional_column->name); column_names.push_back(read_additional_column->name); - table_expression_columns.source_columns_set.emplace(read_additional_column->name); - table_expression_columns.source_input_columns.emplace_back(*read_additional_column); - table_expression_columns.column_name_to_column_identifier.emplace(read_additional_column->name, planner_context.getColumnUniqueIdentifier()); + table_expression_columns.source_columns_names.emplace(read_additional_column->name); + table_expression_columns.source_columns.emplace_back(*read_additional_column); + table_expression_columns.column_name_to_column_identifier.emplace(read_additional_column->name, column_identifier); } if (!column_names.empty()) @@ -1152,18 +1185,11 @@ JoinTreeNodePlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_express rename_actions_dag->getOutputs()[position] = &rename_actions_dag->addAlias(*node_to_rename, column_identifier); } - planner_context.actions_chain.addNode(std::make_unique(rename_actions_dag, true /*available_output_columns_only_aliases*/)); - size_t actions_chain_node_index = planner_context.actions_chain.getLastNodeIndex(); + auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), rename_actions_dag); + rename_step->setStepDescription("Change column names to column identifiers"); + query_plan.addStep(std::move(rename_step)); - QueryPlanBuilder builder(std::move(query_plan)); - - builder.addBuildStep([rename_actions_dag](QueryPlan & build_plan) { - auto rename_step = std::make_unique(build_plan.getCurrentDataStream(), rename_actions_dag); - rename_step->setStepDescription("Change column names to column identifiers"); - build_plan.addStep(std::move(rename_step)); - }); - - return {std::move(builder), {actions_chain_node_index}}; + return query_plan; } class JoinClause @@ -1191,6 +1217,16 @@ public: return right_key_nodes; } + ActionsDAG::NodeRawConstPtrs & getLeftKeyNodes() + { + return left_key_nodes; + } + + ActionsDAG::NodeRawConstPtrs & getRightKeyNodes() + { + return right_key_nodes; + } + const ActionsDAG::NodeRawConstPtrs & getLeftFilterConditionNodes() const { return left_filter_condition_nodes; @@ -1396,11 +1432,7 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & const JoinNode & join_node, const PlannerContext & planner_context) { - std::cout << "buildJoinClausesAndActions " << join_node.formatASTForErrorMessage() << std::endl; - ActionsDAGPtr join_expression_actions = std::make_shared(join_expression_input_columns); - // std::cout << "buildJoinClausesAndActions join expression actions dag before visitor " << std::endl; - // std::cout << join_expression_actions->dumpDAG() << std::endl; QueryTreeActionsVisitor join_expression_visitor(join_expression_actions, planner_context); auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_node.getJoinExpression()); @@ -1409,20 +1441,12 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & "JOIN {} ON clause contains multiple expressions", join_node.formatASTForErrorMessage()); - // std::cout << "buildJoinClausesAndActions join expression actions dag after visitor " << std::endl; - // std::cout << join_expression_actions->dumpDAG() << std::endl; - const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0]; if (!join_expressions_actions_root_node->function) throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "JOIN {} join expression expected function", join_node.formatASTForErrorMessage()); - std::cout << "buildJoinClausesAndActions join expressions actions DAG dump " << std::endl; - std::cout << join_expression_actions->dumpDAG() << std::endl; - - std::cout << "root node " << join_expressions_actions_root_node << std::endl; - size_t left_table_expression_columns_size = left_table_expression_columns.size(); Names join_left_actions_names; @@ -1531,15 +1555,69 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name); } - for (const auto & left_key_node : join_clause.getLeftKeyNodes()) - { - join_expression_actions->addOrReplaceInOutputs(*left_key_node); - add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name); - } + assert(join_clause.getLeftKeyNodes().size() == join_clause.getRightKeyNodes().size()); + size_t join_clause_left_key_nodes_size = join_clause.getLeftKeyNodes().size(); - for (const auto & right_key_node : join_clause.getRightKeyNodes()) + for (size_t i = 0; i < join_clause_left_key_nodes_size; ++i) { + auto & left_key_node = join_clause.getLeftKeyNodes()[i]; + auto & right_key_node = join_clause.getRightKeyNodes()[i]; + + if (!left_key_node->result_type->equals(*right_key_node->result_type)) + { + DataTypePtr common_type; + + try + { + common_type = getLeastSupertype(DataTypes{left_key_node->result_type, right_key_node->result_type}); + } + catch (Exception & ex) + { + ex.addMessage("JOIN {} cannot infer common type in ON section for keys. Left key {} type {}. Right key {} type {}", + join_node.formatASTForErrorMessage(), + left_key_node->result_name, + left_key_node->result_type->getName(), + right_key_node->result_name, + right_key_node->result_type->getName()); + } + + ColumnWithTypeAndName cast_column; + cast_column.name = "__constant_" + common_type->getName(); + cast_column.column = DataTypeString().createColumnConst(0, common_type->getName()); + cast_column.type = std::make_shared(); + + const ActionsDAG::Node * cast_type_constant_node = nullptr; + + if (!left_key_node->result_type->equals(*common_type)) + { + cast_type_constant_node = &join_expression_actions->addColumn(cast_column); + + FunctionCastBase::Diagnostic diagnostic = {left_key_node->result_name, left_key_node->result_name}; + FunctionOverloadResolverPtr func_builder_cast + = CastInternalOverloadResolver::createImpl(diagnostic); + + ActionsDAG::NodeRawConstPtrs children = {left_key_node, cast_type_constant_node}; + left_key_node = &join_expression_actions->addFunction(func_builder_cast, std::move(children), {}); + } + + if (!right_key_node->result_type->equals(*common_type)) + { + if (!cast_type_constant_node) + cast_type_constant_node = &join_expression_actions->addColumn(cast_column); + + FunctionCastBase::Diagnostic diagnostic = {right_key_node->result_name, right_key_node->result_name}; + FunctionOverloadResolverPtr func_builder_cast + = CastInternalOverloadResolver::createImpl(std::move(diagnostic)); + + ActionsDAG::NodeRawConstPtrs children = {right_key_node, cast_type_constant_node}; + right_key_node = &join_expression_actions->addFunction(func_builder_cast, std::move(children), {}); + } + } + + join_expression_actions->addOrReplaceInOutputs(*left_key_node); join_expression_actions->addOrReplaceInOutputs(*right_key_node); + + add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name); add_necessary_name_if_needed(JoinTableSide::Right, right_key_node->result_name); } } @@ -1553,26 +1631,24 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & return result; } -JoinTreeNodePlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, +QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, PlannerContext & planner_context) { auto & join_node = join_tree_node->as(); - auto left_plan_build_result = buildQueryPlanForJoinTreeNode(join_node.getLeftTableExpression(), + auto left_plan = buildQueryPlanForJoinTreeNode(join_node.getLeftTableExpression(), select_query_info, select_query_options, planner_context); - auto left_plan_builder = std::move(left_plan_build_result.plan_builder); - ColumnsWithTypeAndName left_plan_output_columns = planner_context.actions_chain.getAvailableOutputColumns(left_plan_build_result.actions_chain_node_indices); + auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - auto right_plan_build_result = buildQueryPlanForJoinTreeNode(join_node.getRightTableExpression(), + auto right_plan = buildQueryPlanForJoinTreeNode(join_node.getRightTableExpression(), select_query_info, select_query_options, planner_context); - auto right_plan_builder = std::move(right_plan_build_result.plan_builder); - auto right_plan_output_columns = planner_context.actions_chain.getAvailableOutputColumns(right_plan_build_result.actions_chain_node_indices); + auto right_plan_output_columns = right_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); if (join_node.getStrictness() == JoinStrictness::Asof) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, @@ -1581,178 +1657,256 @@ JoinTreeNodePlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, JoinClausesAndActions join_clauses_and_actions; - std::vector actions_chain_node_indices; - std::vector actions_chain_right_plan_node_indexes; - - if (join_node.getJoinExpression()) + if (join_node.isOnJoinExpression()) { - if (join_node.isUsingJoinExpression()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "JOIN {} USING is unsupported", - join_node.formatASTForErrorMessage()); - auto join_expression_input_columns = left_plan_output_columns; join_expression_input_columns.insert(join_expression_input_columns.end(), right_plan_output_columns.begin(), right_plan_output_columns.end()); + join_clauses_and_actions = buildJoinClausesAndActions(join_expression_input_columns, left_plan_output_columns, right_plan_output_columns, join_node, planner_context); - auto left_join_actions_node = std::make_unique(join_clauses_and_actions.left_join_expressions_actions); - left_join_actions_node->addParentIndices(left_plan_build_result.actions_chain_node_indices); - planner_context.actions_chain.addNode(std::move(left_join_actions_node)); - actions_chain_node_indices.push_back(planner_context.actions_chain.getLastNodeIndex()); + auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); + left_join_expressions_actions_step->setStepDescription("Join actions"); + left_plan.addStep(std::move(left_join_expressions_actions_step)); - auto right_join_actions_node = std::make_unique(join_clauses_and_actions.right_join_expressions_actions); - right_join_actions_node->addParentIndices(right_plan_build_result.actions_chain_node_indices); - planner_context.actions_chain.addNode(std::move(right_join_actions_node)); - actions_chain_node_indices.push_back(planner_context.actions_chain.getLastNodeIndex()); - actions_chain_right_plan_node_indexes.push_back(planner_context.actions_chain.getLastNodeIndex()); - - left_plan_builder.addBuildStep([left_join_expressions_actions = join_clauses_and_actions.left_join_expressions_actions](QueryPlan & build_plan) - { - auto left_join_expressions_actions_step = std::make_unique(build_plan.getCurrentDataStream(), left_join_expressions_actions); - left_join_expressions_actions_step->setStepDescription("Join actions"); - build_plan.addStep(std::move(left_join_expressions_actions_step)); - }); - - right_plan_builder.addBuildStep([right_join_expressions_actions = join_clauses_and_actions.right_join_expressions_actions](QueryPlan & build_plan) - { - auto right_join_expressions_actions_step = std::make_unique(build_plan.getCurrentDataStream(), right_join_expressions_actions); - right_join_expressions_actions_step->setStepDescription("Join actions"); - build_plan.addStep(std::move(right_join_expressions_actions_step)); - }); - } - else - { - actions_chain_right_plan_node_indexes = right_plan_build_result.actions_chain_node_indices; - actions_chain_node_indices.insert(actions_chain_node_indices.end(), actions_chain_right_plan_node_indexes.begin(), actions_chain_right_plan_node_indexes.end()); + auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); + right_join_expressions_actions_step->setStepDescription("Join actions"); + right_plan.addStep(std::move(right_join_expressions_actions_step)); } - std::vector builders; - builders.emplace_back(std::move(left_plan_builder)); - builders.emplace_back(std::move(right_plan_builder)); + std::unordered_map left_plan_column_name_to_cast_type; + std::unordered_map right_plan_column_name_to_cast_type; - QueryPlanBuilder builder(std::move(builders), [join_clauses_and_actions, actions_chain_right_plan_node_indexes, &join_node, &planner_context](std::vector build_query_plans) + if (join_node.isUsingJoinExpression()) { - if (build_query_plans.size() != 2) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Join step expects 2 query plans. Actual {}", build_query_plans.size()); - - auto left_plan = std::move(build_query_plans[0]); - auto right_plan = std::move(build_query_plans[1]); - - auto table_join = std::make_shared(); - table_join->getTableJoin() = join_node.toASTTableJoin()->as(); - if (join_node.getKind() == JoinKind::Comma) - table_join->getTableJoin().kind = JoinKind::Cross; - table_join->getTableJoin().strictness = JoinStrictness::All; - - NameSet join_clauses_right_column_names; - - if (join_node.getJoinExpression()) + auto & join_node_using_columns_list = join_node.getJoinExpression()->as(); + for (auto & join_node_using_node : join_node_using_columns_list.getNodes()) { - const auto & join_clauses = join_clauses_and_actions.join_clauses; - auto & table_join_clauses = table_join->getClauses(); + auto & join_node_using_column_node = join_node_using_node->as(); + auto & inner_columns_list = join_node_using_column_node.getExpressionOrThrow()->as(); - for (const auto & join_clause : join_clauses) + auto & left_inner_column_node = inner_columns_list.getNodes().at(0); + auto & left_inner_column = left_inner_column_node->as(); + + auto & right_inner_column_node = inner_columns_list.getNodes().at(1); + auto & right_inner_column = right_inner_column_node->as(); + + const auto & join_node_using_column_node_type = join_node_using_column_node.getColumnType(); + if (!left_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) { - table_join_clauses.emplace_back(); - auto & table_join_clause = table_join_clauses.back(); + auto left_inner_column_identifier = planner_context.getColumnIdentifierOrThrow(left_inner_column_node.get()); + left_plan_column_name_to_cast_type.emplace(left_inner_column_identifier, join_node_using_column_node_type); + } - const auto & join_clause_left_key_nodes = join_clause.getLeftKeyNodes(); - const auto & join_clause_right_key_nodes = join_clause.getRightKeyNodes(); - - size_t join_clause_key_nodes_size = join_clause_left_key_nodes.size(); - assert(join_clause_key_nodes_size == join_clause_right_key_nodes.size()); - - for (size_t i = 0; i < join_clause_key_nodes_size; ++i) - { - table_join_clause.key_names_left.push_back(join_clause_left_key_nodes[i]->result_name); - table_join_clause.key_names_right.push_back(join_clause_right_key_nodes[i]->result_name); - join_clauses_right_column_names.insert(join_clause_right_key_nodes[i]->result_name); - } - - const auto & join_clause_get_left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); - if (!join_clause_get_left_filter_condition_nodes.empty()) - { - if (join_clause_get_left_filter_condition_nodes.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "JOIN {} left filter conditions size must be 1. Actual {}", - join_node.formatASTForErrorMessage(), - join_clause_get_left_filter_condition_nodes.size()); - - const auto & join_clause_left_filter_condition_name = join_clause_get_left_filter_condition_nodes[0]->result_name; - table_join_clause.analyzer_left_filter_condition_column_name = join_clause_left_filter_condition_name; - } - - const auto & join_clause_get_right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); - if (!join_clause_get_right_filter_condition_nodes.empty()) - { - if (join_clause_get_right_filter_condition_nodes.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "JOIN {} right filter conditions size must be 1. Actual {}", - join_node.formatASTForErrorMessage(), - join_clause_get_right_filter_condition_nodes.size()); - - const auto & join_clause_right_filter_condition_name = join_clause_get_right_filter_condition_nodes[0]->result_name; - table_join_clause.analyzer_right_filter_condition_column_name = join_clause_right_filter_condition_name; - join_clauses_right_column_names.insert(join_clause_right_filter_condition_name); - } + if (!right_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) + { + auto right_inner_column_identifier = planner_context.getColumnIdentifierOrThrow(right_inner_column_node.get()); + right_plan_column_name_to_cast_type.emplace(right_inner_column_identifier, join_node_using_column_node_type); } } + } - auto left_table_names = left_plan.getCurrentDataStream().header.getNames(); - NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); + auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map & plan_column_name_to_cast_type) + { + auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); - auto columns_from_joined_table = right_plan.getCurrentDataStream().header.getNamesAndTypesList(); - table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); - - NamesAndTypesList columns_added_by_join; - for (auto & column_from_joined_table : columns_from_joined_table) + for (auto & output_node : cast_actions_dag->getOutputs()) { - for (const auto & actions_chain_right_plan_node_index : actions_chain_right_plan_node_indexes) - { - const auto & child_required_ouput_columns_names = planner_context.actions_chain[actions_chain_right_plan_node_index]->getChildRequiredOutputColumnsNames(); + auto it = plan_column_name_to_cast_type.find(output_node->result_name); + if (it == plan_column_name_to_cast_type.end()) + continue; - if (child_required_ouput_columns_names.contains(column_from_joined_table.name)) - { - columns_added_by_join.insert(columns_added_by_join.end(), column_from_joined_table); - break; - } - } + const auto & cast_type = it->second; + auto cast_type_name = cast_type->getName(); + + ColumnWithTypeAndName column; + column.name = "__constant_" + cast_type_name; + column.column = DataTypeString().createColumnConst(0, cast_type_name); + column.type = std::make_shared(); + + const auto * cast_type_constant_node = &cast_actions_dag->addColumn(std::move(column)); + + FunctionCastBase::Diagnostic diagnostic = {output_node->result_name, output_node->result_name}; + FunctionOverloadResolverPtr func_builder_cast + = CastInternalOverloadResolver::createImpl(std::move(diagnostic)); + + ActionsDAG::NodeRawConstPtrs children = {output_node, cast_type_constant_node}; + output_node = &cast_actions_dag->addFunction(func_builder_cast, std::move(children), output_node->result_name); } - table_join->setColumnsAddedByJoin(columns_added_by_join); + auto cast_join_columns_step + = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); + cast_join_columns_step->setStepDescription("Cast JOIN USING columns"); + plan_to_add_cast.addStep(std::move(cast_join_columns_step)); + }; - size_t max_block_size = planner_context.query_context->getSettingsRef().max_block_size; - size_t max_streams = planner_context.query_context->getSettingsRef().max_threads; + if (!left_plan_column_name_to_cast_type.empty()) + join_cast_plan_output_nodes(left_plan, left_plan_column_name_to_cast_type); - JoinPtr join_ptr = std::make_shared(table_join, right_plan.getCurrentDataStream().header, false /*any_take_last_row*/); - QueryPlanStepPtr join_step = std::make_unique( - left_plan.getCurrentDataStream(), - right_plan.getCurrentDataStream(), - join_ptr, - max_block_size, - max_streams, - false /*optimize_read_in_order*/); + if (!right_plan_column_name_to_cast_type.empty()) + join_cast_plan_output_nodes(right_plan, right_plan_column_name_to_cast_type); - join_step->setStepDescription(fmt::format("JOIN {}", JoinPipelineType::FillRightFirst)); + JoinKind join_kind = join_node.getKind(); + bool join_use_nulls = planner_context.query_context->getSettingsRef().join_use_nulls; + auto to_nullable_function = FunctionFactory::instance().get("toNullable", planner_context.query_context); - std::vector plans; - plans.emplace_back(std::make_unique(std::move(left_plan))); - plans.emplace_back(std::make_unique(std::move(right_plan))); + auto join_cast_plan_columns_to_nullable = [&](QueryPlan & plan_to_add_cast) + { + auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); - auto result = QueryPlan(); - result.unitePlans(std::move(join_step), {std::move(plans)}); + for (auto & output_node : cast_actions_dag->getOutputs()) + { + if (output_node->type == ActionsDAG::ActionType::INPUT && output_node->result_name.starts_with("__column")) + output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); + } - return result; - }); + auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); + cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable"); + plan_to_add_cast.addStep(std::move(cast_join_columns_step)); + }; - return {std::move(builder), actions_chain_node_indices}; + if (join_use_nulls) + { + if (isFull(join_kind)) + { + join_cast_plan_columns_to_nullable(left_plan); + join_cast_plan_columns_to_nullable(right_plan); + } + else if (isLeft(join_kind)) + { + join_cast_plan_columns_to_nullable(right_plan); + } + else if (isRight(join_kind)) + { + join_cast_plan_columns_to_nullable(left_plan); + } + } + + auto table_join = std::make_shared(); + table_join->getTableJoin() = join_node.toASTTableJoin()->as(); + if (join_node.getKind() == JoinKind::Comma) + table_join->getTableJoin().kind = JoinKind::Cross; + table_join->getTableJoin().strictness = JoinStrictness::All; + + if (join_node.isOnJoinExpression()) + { + const auto & join_clauses = join_clauses_and_actions.join_clauses; + auto & table_join_clauses = table_join->getClauses(); + + for (const auto & join_clause : join_clauses) + { + table_join_clauses.emplace_back(); + auto & table_join_clause = table_join_clauses.back(); + + const auto & join_clause_left_key_nodes = join_clause.getLeftKeyNodes(); + const auto & join_clause_right_key_nodes = join_clause.getRightKeyNodes(); + + size_t join_clause_key_nodes_size = join_clause_left_key_nodes.size(); + assert(join_clause_key_nodes_size == join_clause_right_key_nodes.size()); + + for (size_t i = 0; i < join_clause_key_nodes_size; ++i) + { + table_join_clause.key_names_left.push_back(join_clause_left_key_nodes[i]->result_name); + table_join_clause.key_names_right.push_back(join_clause_right_key_nodes[i]->result_name); + } + + const auto & join_clause_get_left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); + if (!join_clause_get_left_filter_condition_nodes.empty()) + { + if (join_clause_get_left_filter_condition_nodes.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} left filter conditions size must be 1. Actual {}", + join_node.formatASTForErrorMessage(), + join_clause_get_left_filter_condition_nodes.size()); + + const auto & join_clause_left_filter_condition_name = join_clause_get_left_filter_condition_nodes[0]->result_name; + table_join_clause.analyzer_left_filter_condition_column_name = join_clause_left_filter_condition_name; + } + + const auto & join_clause_get_right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); + if (!join_clause_get_right_filter_condition_nodes.empty()) + { + if (join_clause_get_right_filter_condition_nodes.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} right filter conditions size must be 1. Actual {}", + join_node.formatASTForErrorMessage(), + join_clause_get_right_filter_condition_nodes.size()); + + const auto & join_clause_right_filter_condition_name = join_clause_get_right_filter_condition_nodes[0]->result_name; + table_join_clause.analyzer_right_filter_condition_column_name = join_clause_right_filter_condition_name; + } + } + } + else if (join_node.isUsingJoinExpression()) + { + auto & table_join_clauses = table_join->getClauses(); + table_join_clauses.emplace_back(); + auto & table_join_clause = table_join_clauses.back(); + + auto & using_list = join_node.getJoinExpression()->as(); + + for (auto & join_using_node : using_list.getNodes()) + { + auto & join_using_column_node = join_using_node->as(); + if (!join_using_column_node.getExpression() || + join_using_column_node.getExpression()->getNodeType() != QueryTreeNodeType::LIST) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} column in USING does not have inner columns", + join_node.formatASTForErrorMessage()); + + auto & using_join_columns_list = join_using_column_node.getExpression()->as(); + auto & using_join_left_join_column_node = using_join_columns_list.getNodes().at(0); + auto & using_join_right_join_column_node = using_join_columns_list.getNodes().at(1); + + auto left_column_identifier_it = planner_context.column_node_to_column_identifier.find(using_join_left_join_column_node.get()); + auto right_column_identifier_it = planner_context.column_node_to_column_identifier.find(using_join_right_join_column_node.get()); + + table_join_clause.key_names_left.push_back(left_column_identifier_it->second); + table_join_clause.key_names_right.push_back(right_column_identifier_it->second); + } + } + + auto left_table_names = left_plan.getCurrentDataStream().header.getNames(); + NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); + + auto columns_from_joined_table = right_plan.getCurrentDataStream().header.getNamesAndTypesList(); + table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); + + + for (auto & column_from_joined_table : columns_from_joined_table) + { + if (column_from_joined_table.name.starts_with("__column")) + table_join->addJoinedColumn(column_from_joined_table); + } + + size_t max_block_size = planner_context.query_context->getSettingsRef().max_block_size; + size_t max_streams = planner_context.query_context->getSettingsRef().max_threads; + + JoinPtr join_ptr = std::make_shared(table_join, right_plan.getCurrentDataStream().header, false /*any_take_last_row*/); + QueryPlanStepPtr join_step = std::make_unique( + left_plan.getCurrentDataStream(), + right_plan.getCurrentDataStream(), + join_ptr, + max_block_size, + max_streams, + false /*optimize_read_in_order*/); + + join_step->setStepDescription(fmt::format("JOIN {}", JoinPipelineType::FillRightFirst)); + + std::vector plans; + plans.emplace_back(std::make_unique(std::move(left_plan))); + plans.emplace_back(std::make_unique(std::move(right_plan))); + + auto result_plan = QueryPlan(); + result_plan.unitePlans(std::move(join_step), {std::move(plans)}); + + return result_plan; } -JoinTreeNodePlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, +QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, PlannerContext & planner_context) @@ -1881,44 +2035,47 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() PlannerContext planner_context; planner_context.query_context = getContext(); + CollectTableExpressionIdentifiersVisitor collect_table_expression_identifiers_visitor; + collect_table_expression_identifiers_visitor.visit(query_node.getFrom(), planner_context); + CollectSourceColumnsVisitor::Data data {planner_context}; CollectSourceColumnsVisitor collect_source_columns_visitor(data); collect_source_columns_visitor.visit(query_tree); - JoinTreeNodePlan join_tree_node_plan = buildQueryPlanForJoinTreeNode(query_node.getFrom(), select_query_info, select_query_options, planner_context); - auto query_plan_builder = std::move(join_tree_node_plan.plan_builder); - auto action_chain_node_parent_indices = join_tree_node_plan.actions_chain_node_indices; + query_plan = buildQueryPlanForJoinTreeNode(query_node.getFrom(), select_query_info, select_query_options, planner_context); + std::optional> action_chain_node_parent_indices; if (query_node.hasWhere()) { - ColumnsWithTypeAndName where_input = planner_context.actions_chain.getAvailableOutputColumns(action_chain_node_parent_indices); + ColumnsWithTypeAndName where_input; + if (action_chain_node_parent_indices) + planner_context.actions_chain.getAvailableOutputColumns(*action_chain_node_parent_indices); + else + where_input = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + planner_context.where_actions = convertExpressionNodeIntoDAG(query_node.getWhere(), where_input, planner_context); planner_context.where_action_node_name = planner_context.where_actions->getOutputs().at(0)->result_name; auto where_actions_node = std::make_unique(planner_context.where_actions); - where_actions_node->addParentIndices(action_chain_node_parent_indices); + if (action_chain_node_parent_indices) + where_actions_node->addParentIndices(*action_chain_node_parent_indices); + planner_context.actions_chain.addNode(std::move(where_actions_node)); action_chain_node_parent_indices = {planner_context.actions_chain.getLastNodeIndex()}; - - size_t where_node_index = planner_context.actions_chain.size(); - - query_plan_builder.addBuildStep([&, where_node_index](QueryPlan & build_plan) - { - bool remove_filter = !planner_context.actions_chain.at(where_node_index)->getChildRequiredOutputColumnsNames().contains(planner_context.where_action_node_name); - auto where_step = std::make_unique(build_plan.getCurrentDataStream(), - planner_context.where_actions, - planner_context.where_action_node_name, - remove_filter); - where_step->setStepDescription("WHERE"); - build_plan.addStep(std::move(where_step)); - }); + planner_context.where_actions_chain_node_index = planner_context.actions_chain.size(); } - ColumnsWithTypeAndName projection_input = planner_context.actions_chain.getAvailableOutputColumns(action_chain_node_parent_indices); + ColumnsWithTypeAndName projection_input; + if (action_chain_node_parent_indices) + planner_context.actions_chain.getAvailableOutputColumns(*action_chain_node_parent_indices); + else + projection_input = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + planner_context.projection_actions = convertExpressionNodeIntoDAG(query_node.getProjectionNode(), projection_input, planner_context); auto projection_actions_node = std::make_unique(planner_context.projection_actions); - projection_actions_node->addParentIndices(action_chain_node_parent_indices); + if (action_chain_node_parent_indices) + projection_actions_node->addParentIndices(*action_chain_node_parent_indices); planner_context.actions_chain.addNode(std::move(projection_actions_node)); const auto & projection_action_dag_nodes = planner_context.projection_actions->getOutputs(); @@ -1950,12 +2107,24 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() planner_context.projection_actions->project(projection_names); - query_plan_builder.addBuildStep([&](QueryPlan & build_plan) + if (query_node.hasWhere()) { - auto projection_step = std::make_unique(build_plan.getCurrentDataStream(), planner_context.projection_actions); - projection_step->setStepDescription("Projection"); - build_plan.addStep(std::move(projection_step)); - }); + auto & where_actions_chain_node = planner_context.actions_chain.at(planner_context.where_actions_chain_node_index); + bool remove_filter = !where_actions_chain_node->getChildRequiredOutputColumnsNames().contains(planner_context.where_action_node_name); + auto where_step = std::make_unique(query_plan.getCurrentDataStream(), + planner_context.where_actions, + planner_context.where_action_node_name, + remove_filter); + where_step->setStepDescription("WHERE"); + query_plan.addStep(std::move(where_step)); + } + + // std::cout << "Query plan dump" << std::endl; + // std::cout << dumpQueryPlan(query_plan) << std::endl; + + auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), planner_context.projection_actions); + projection_step->setStepDescription("Projection"); + query_plan.addStep(std::move(projection_step)); // std::cout << "Chain dump before finalize" << std::endl; // std::cout << planner_context.actions_chain.dump() << std::endl; @@ -1964,8 +2133,6 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() // std::cout << "Chain dump after finalize" << std::endl; // std::cout << planner_context.actions_chain.dump() << std::endl; - - query_plan = std::move(query_plan_builder).buildPlan(); } } diff --git a/tests/queries/0_stateless/02372_analyzer_join.reference b/tests/queries/0_stateless/02372_analyzer_join.reference new file mode 100644 index 00000000000..17077dfd6eb --- /dev/null +++ b/tests/queries/0_stateless/02372_analyzer_join.reference @@ -0,0 +1,740 @@ +-- { echoOn } + +SELECT 'JOIN INNER'; +JOIN INNER +SELECT 'Join without ON conditions'; +Join without ON conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 +SELECT '--'; +-- +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id + INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id + INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id + INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +SELECT id FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } +SELECT value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } +SELECT 'Join with ON conditions'; +Join with ON conditions +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' + INNER JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT 'Join only join expression use keys'; +Join only join expression use keys +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +SELECT '--'; +-- +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id +INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +SELECT 'Join multiple clauses'; +Join multiple clauses +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id OR t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_0 Join_2_Value_0 Join_3_Value_1 +Join_1_Value_0 Join_2_Value_0 Join_3_Value_2 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_2 +SELECT 'Join expression aliases'; +Join expression aliases +SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1_id = t2_id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +SELECT 'JOIN LEFT'; +JOIN LEFT +SELECT 'Join without ON conditions'; +Join without ON conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value +FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_3 0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_3 0 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 +2 2 Join_1_Value_3 Join_1_Value_3 0 0 +SELECT '--'; +-- +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id + LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_3 0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id + LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_3 0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id + LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +2 2 Join_1_Value_3 Join_1_Value_3 0 0 0 0 Join_3_Value_0 Join_3_Value_0 +SELECT id FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } +SELECT value FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } +SELECT 'Join with ON conditions'; +Join with ON conditions +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 0 +2 Join_1_Value_3 0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 0 +2 Join_1_Value_3 0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 0 +2 Join_1_Value_3 0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 Join_3_Value_0 +2 Join_1_Value_3 0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_3 0 0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +LEFT JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_3 0 0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +LEFT JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_3 0 0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' + LEFT JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 Join_3_Value_0 +2 Join_1_Value_3 0 0 Join_3_Value_0 +SELECT 'Join only join expression use keys'; +Join only join expression use keys +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +Join_1_Value_3 +SELECT '--'; +-- +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +Join_1_Value_3 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_3 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id +LEFT JOIN test_table_join_3 AS t3 ON t1.id = t3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_3 +SELECT 'Join multiple clauses'; +Join multiple clauses +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_3 0 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +LEFT JOIN test_table_join_3 AS t3 ON t1.id = t3.id OR t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_0 Join_2_Value_0 Join_3_Value_1 +Join_1_Value_0 Join_2_Value_0 Join_3_Value_2 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_2 +Join_1_Value_3 +SELECT 'Join expression aliases'; +Join expression aliases +SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_3 0 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1_id = t2_id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_3 0 +SELECT 'JOIN RIGHT'; +JOIN RIGHT +SELECT 'Join without ON conditions'; +Join without ON conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value +FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 +0 0 3 3 Join_2_Value_2 Join_2_Value_2 +SELECT '--'; +-- +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id + RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id + RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id + RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +0 0 0 0 4 4 Join_3_Value_2 Join_3_Value_2 +SELECT id FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } +SELECT value FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } +SELECT 'Join with ON conditions'; +Join with ON conditions +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +0 1 Join_2_Value_1 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +0 1 Join_2_Value_1 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +0 1 Join_2_Value_1 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +RIGHT JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 1 Join_2_Value_1 0 Join_3_Value_0 +0 3 Join_2_Value_2 0 Join_3_Value_0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +RIGHT JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' + RIGHT JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_2 +SELECT 'Join only join expression use keys'; +Join only join expression use keys +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 + Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 + Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 + Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id +RIGHT JOIN test_table_join_3 AS t3 ON t1.id = t3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 + Join_2_Value_2 Join_3_Value_0 + Join_3_Value_2 +SELECT 'Join multiple clauses'; +Join multiple clauses +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +RIGHT JOIN test_table_join_3 AS t3 ON t1.id = t3.id OR t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_0 Join_2_Value_0 Join_3_Value_1 +Join_1_Value_0 Join_2_Value_0 Join_3_Value_2 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_2 + Join_2_Value_2 Join_3_Value_0 +SELECT 'Join expression aliases'; +Join expression aliases +SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1_id = t2_id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +0 3 Join_2_Value_2 +SELECT 'JOIN FULL'; +JOIN FULL +SELECT 'Join without ON conditions'; +Join without ON conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value +FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_3 0 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_3 0 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 +2 2 Join_1_Value_3 Join_1_Value_3 0 0 +0 0 3 3 Join_2_Value_2 Join_2_Value_2 +SELECT '--'; +-- +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id + FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_3 0 0 Join_3_Value_0 +0 3 Join_2_Value_2 0 +0 0 4 Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id + FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_3 0 0 Join_3_Value_0 +0 3 Join_2_Value_2 0 +0 0 4 Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id + FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +2 2 Join_1_Value_3 Join_1_Value_3 0 0 0 0 Join_3_Value_0 Join_3_Value_0 +0 0 3 3 Join_2_Value_2 Join_2_Value_2 0 0 +0 0 0 0 4 4 Join_3_Value_2 Join_3_Value_2 +SELECT id FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } +SELECT value FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } +SELECT 'Join with ON conditions'; +Join with ON conditions +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 0 +2 Join_1_Value_3 0 +0 1 Join_2_Value_1 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 0 +2 Join_1_Value_3 0 +0 1 Join_2_Value_1 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 0 +2 Join_1_Value_3 0 +0 1 Join_2_Value_1 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 Join_3_Value_0 +2 Join_1_Value_3 0 0 Join_3_Value_0 +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_2 0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_3 0 0 +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_2 0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +FULL JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_3 0 0 +0 1 Join_2_Value_1 0 Join_3_Value_0 +0 3 Join_2_Value_2 0 Join_3_Value_0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +FULL JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_3 0 0 +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_2 0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' + FULL JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 Join_3_Value_0 +2 Join_1_Value_3 0 0 Join_3_Value_0 +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_2 0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_2 +SELECT 'Join only join expression use keys'; +Join only join expression use keys +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +Join_1_Value_3 + Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +Join_1_Value_3 + Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_3 Join_3_Value_0 + Join_2_Value_2 + Join_3_Value_2 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id +FULL JOIN test_table_join_3 AS t3 ON t1.id = t3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_3 + Join_2_Value_2 Join_3_Value_0 + Join_3_Value_2 +SELECT 'Join multiple clauses'; +Join multiple clauses +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_3 0 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +FULL JOIN test_table_join_3 AS t3 ON t1.id = t3.id OR t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_0 Join_2_Value_0 Join_3_Value_1 +Join_1_Value_0 Join_2_Value_0 Join_3_Value_2 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_2 +Join_1_Value_3 + Join_2_Value_2 Join_3_Value_0 +SELECT 'Join expression aliases'; +Join expression aliases +SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_3 0 +0 3 Join_2_Value_2 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_3 0 +0 3 Join_2_Value_2 diff --git a/tests/queries/0_stateless/02372_analyzer_join.sql.j2 b/tests/queries/0_stateless/02372_analyzer_join.sql.j2 new file mode 100644 index 00000000000..966c96253b5 --- /dev/null +++ b/tests/queries/0_stateless/02372_analyzer_join.sql.j2 @@ -0,0 +1,180 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table_join_1; +CREATE TABLE test_table_join_1 +( + id UInt64, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_2; +CREATE TABLE test_table_join_2 +( + id UInt64, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_3; +CREATE TABLE test_table_join_3 +( + id UInt64, + value String +) ENGINE = TinyLog; + +INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value_0'); +INSERT INTO test_table_join_1 VALUES (1, 'Join_1_Value_1'); +INSERT INTO test_table_join_1 VALUES (2, 'Join_1_Value_3'); + +INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value_0'); +INSERT INTO test_table_join_2 VALUES (1, 'Join_2_Value_1'); +INSERT INTO test_table_join_2 VALUES (3, 'Join_2_Value_2'); + +INSERT INTO test_table_join_3 VALUES (0, 'Join_3_Value_0'); +INSERT INTO test_table_join_3 VALUES (1, 'Join_3_Value_1'); +INSERT INTO test_table_join_3 VALUES (4, 'Join_3_Value_2'); + +-- { echoOn } + +{% for join_type in ['INNER', 'LEFT', 'RIGHT', 'FULL'] -%} + +SELECT 'JOIN {{ join_type }}'; + +SELECT 'Join without ON conditions'; + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value +FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id + {{ join_type }} JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id + {{ join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id + {{ join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id; + +SELECT id FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } + +SELECT value FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } + +SELECT 'Join with ON conditions'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +{{ join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +{{ join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +{{ join_type }} JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +{{ join_type }} JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' + {{ join_type }} JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; + +SELECT 'Join only join expression use keys'; + +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +{{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id +{{ join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id; + +SELECT '--'; + +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +{{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id +{{ join_type }} JOIN test_table_join_3 AS t3 ON t1.id = t3.id; + +SELECT 'Join multiple clauses'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; + +SELECT '--'; + +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +{{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +{{ join_type }} JOIN test_table_join_3 AS t3 ON t1.id = t3.id OR t1.id = t2.id; + +SELECT 'Join expression aliases'; + +SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); + +SELECT '--'; + +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1_id = t2_id; + +{% endfor %} + +-- { echoOff } + +DROP TABLE test_table_join_1; +DROP TABLE test_table_join_2; +DROP TABLE test_table_join_3; diff --git a/tests/queries/0_stateless/02372_analyzer_join_inner.reference b/tests/queries/0_stateless/02372_analyzer_join_inner.reference deleted file mode 100644 index 3f8435c35d4..00000000000 --- a/tests/queries/0_stateless/02372_analyzer_join_inner.reference +++ /dev/null @@ -1,51 +0,0 @@ -Join without ON conditions -0 Join_1_Value_0 0 Join_2_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 --- -0 Join_1_Value_0 0 Join_2_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 --- -0 Join_1_Value_0 0 Join_2_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 --- -0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 -1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 --- -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 --- -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 --- -0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 -1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 -Join with ON conditions -0 Join_1_Value_0 0 Join_2_Value_0 --- -0 Join_1_Value_0 0 Join_2_Value_0 --- -0 Join_1_Value_0 0 Join_2_Value_0 --- -0 Join_1_Value_0 0 Join_2_Value_0 --- -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 --- -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 --- -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 --- -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 --- -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -Join only join expression use keys -Join_1_Value_0 Join_2_Value_0 -Join_1_Value_1 Join_2_Value_1 --- -Join_1_Value_0 Join_2_Value_0 -Join_1_Value_1 Join_2_Value_1 --- -Join_1_Value_0 Join_2_Value_0 Join_2_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_2_Value_1 --- -Join_1_Value_0 Join_2_Value_0 Join_2_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_2_Value_1 diff --git a/tests/queries/0_stateless/02372_analyzer_join_inner.sql b/tests/queries/0_stateless/02372_analyzer_join_inner.sql deleted file mode 100644 index 5c915deef56..00000000000 --- a/tests/queries/0_stateless/02372_analyzer_join_inner.sql +++ /dev/null @@ -1,156 +0,0 @@ -SET use_analyzer = 1; - -DROP TABLE IF EXISTS test_table_join_1; -CREATE TABLE test_table_join_1 -( - id UInt64, - value String -) ENGINE = TinyLog; - -DROP TABLE IF EXISTS test_table_join_2; -CREATE TABLE test_table_join_2 -( - id UInt64, - value String -) ENGINE = TinyLog; - -DROP TABLE IF EXISTS test_table_join_3; -CREATE TABLE test_table_join_3 -( - id UInt64, - value String -) ENGINE = TinyLog; - -INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value_0'); -INSERT INTO test_table_join_1 VALUES (1, 'Join_1_Value_1'); -INSERT INTO test_table_join_1 VALUES (3, 'Join_1_Value_3'); - -INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value_0'); -INSERT INTO test_table_join_2 VALUES (1, 'Join_2_Value_1'); -INSERT INTO test_table_join_2 VALUES (2, 'Join_2_Value_2'); - -INSERT INTO test_table_join_3 VALUES (0, 'Join_3_Value_0'); -INSERT INTO test_table_join_3 VALUES (1, 'Join_3_Value_1'); -INSERT INTO test_table_join_3 VALUES (2, 'Join_3_Value_2'); - -SELECT 'Join without ON conditions'; - -SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value -FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; - -SELECT '--'; - -SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; - -SELECT '--'; - -SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; - -SELECT '--'; - -SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value -FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id - INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id - INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id; - -SELECT '--'; - -SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, -t3.id, test_table_join_3.id, t3.value, test_table_join_3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id - INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id; - -SELECT id FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } - -SELECT value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } - -SELECT 'Join with ON conditions'; - -SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' - INNER JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; - -SELECT 'Join only join expression use keys'; - -SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; - -SELECT '--'; - -SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; - -SELECT '--'; - -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id -INNER JOIN test_table_join_2 AS t3 ON t2.id = t3.id; - -SELECT '--'; - -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id -INNER JOIN test_table_join_2 AS t3 ON t1.id = t3.id; - - -DROP TABLE test_table_join_1; -DROP TABLE test_table_join_2; -DROP TABLE test_table_join_3; From fd101281c8292007ad283368b0884abcc1676904 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 20 Aug 2022 16:32:00 +0200 Subject: [PATCH 068/188] Added join_use_nulls test --- .../02373_analyzer_join_use_nulls.reference | 56 +++++++++++++++ .../02373_analyzer_join_use_nulls.sql | 69 +++++++++++++++++++ 2 files changed, 125 insertions(+) create mode 100644 tests/queries/0_stateless/02373_analyzer_join_use_nulls.reference create mode 100644 tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql diff --git a/tests/queries/0_stateless/02373_analyzer_join_use_nulls.reference b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.reference new file mode 100644 index 00000000000..20c5c1c464f --- /dev/null +++ b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.reference @@ -0,0 +1,56 @@ +-- { echoOn } + +SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String +1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String +SELECT '--'; +-- +SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +0 UInt64 Join_1_Value_0 String 0 Nullable(UInt64) Join_2_Value_0 Nullable(String) +1 UInt64 Join_1_Value_1 String 1 Nullable(UInt64) Join_2_Value_1 Nullable(String) +2 UInt64 Join_1_Value_3 String \N Nullable(UInt64) \N Nullable(String) +SELECT '--'; +-- +SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 UInt64 Join_2_Value_0 String +1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 UInt64 Join_2_Value_1 String +\N Nullable(UInt64) \N Nullable(String) 3 UInt64 Join_2_Value_2 String +SELECT '--'; +-- +SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 Nullable(UInt64) Join_2_Value_0 Nullable(String) +1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 Nullable(UInt64) Join_2_Value_1 Nullable(String) +2 Nullable(UInt64) Join_1_Value_3 Nullable(String) \N Nullable(UInt64) \N Nullable(String) +\N Nullable(UInt64) \N Nullable(String) 3 Nullable(UInt64) Join_2_Value_2 Nullable(String) +SELECT '--'; +-- +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String +SELECT '--'; +-- +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 Nullable(UInt64) Join_2_Value_0 Nullable(String) +1 UInt64 1 UInt64 Join_1_Value_1 String 1 Nullable(UInt64) Join_2_Value_1 Nullable(String) +2 UInt64 2 UInt64 Join_1_Value_3 String \N Nullable(UInt64) \N Nullable(String) +SELECT '--'; +-- +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id); +0 UInt64 0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 UInt64 Join_2_Value_0 String +1 UInt64 1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 UInt64 Join_2_Value_1 String +3 UInt64 \N Nullable(UInt64) \N Nullable(String) 3 UInt64 Join_2_Value_2 String +SELECT '--'; +-- +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id); +0 Nullable(UInt64) 0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 Nullable(UInt64) Join_2_Value_0 Nullable(String) +1 Nullable(UInt64) 1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 Nullable(UInt64) Join_2_Value_1 Nullable(String) +2 Nullable(UInt64) 2 Nullable(UInt64) Join_1_Value_3 Nullable(String) \N Nullable(UInt64) \N Nullable(String) +\N Nullable(UInt64) \N Nullable(UInt64) \N Nullable(String) 3 Nullable(UInt64) Join_2_Value_2 Nullable(String) diff --git a/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql new file mode 100644 index 00000000000..1d07589dbe9 --- /dev/null +++ b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql @@ -0,0 +1,69 @@ +SET use_analyzer = 1; +SET join_use_nulls = 1; + +DROP TABLE IF EXISTS test_table_join_1; +CREATE TABLE test_table_join_1 +( + id UInt64, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_2; +CREATE TABLE test_table_join_2 +( + id UInt64, + value String +) ENGINE = TinyLog; + +INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value_0'); +INSERT INTO test_table_join_1 VALUES (1, 'Join_1_Value_1'); +INSERT INTO test_table_join_1 VALUES (2, 'Join_1_Value_3'); + +INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value_0'); +INSERT INTO test_table_join_2 VALUES (1, 'Join_2_Value_1'); +INSERT INTO test_table_join_2 VALUES (3, 'Join_2_Value_2'); + +-- { echoOn } + +SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id); + +SELECT '--'; + +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id); + +SELECT '--'; + +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id); + +SELECT '--'; + +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id); + +-- { echoOff } + +DROP TABLE test_table_join_1; +DROP TABLE test_table_join_2; From 89e5b09e7fbf522b63c2d51f6625c88340478a34 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 20 Aug 2022 22:15:33 +0200 Subject: [PATCH 069/188] Updated JOIN tests --- src/Analyzer/QueryAnalysisPass.cpp | 61 +- .../InterpreterSelectQueryAnalyzer.cpp | 18 +- .../0_stateless/02371_analyzer_join_cross.sql | 4 +- .../0_stateless/02372_analyzer_join.reference | 1848 ++++++++++++----- .../0_stateless/02372_analyzer_join.sql.j2 | 160 +- .../02373_analyzer_join_use_nulls.reference | 28 +- .../02373_analyzer_join_use_nulls.sql | 16 +- .../02374_analyzer_join_using.reference | 452 ++++ .../02374_analyzer_join_using.sql.j2 | 87 + 9 files changed, 2027 insertions(+), 647 deletions(-) create mode 100644 tests/queries/0_stateless/02374_analyzer_join_using.reference create mode 100644 tests/queries/0_stateless/02374_analyzer_join_using.sql.j2 diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 7b6fd4b3bf4..d33ed683799 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -86,6 +86,7 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int INCORRECT_ELEMENT_OF_SET; extern const int TYPE_MISMATCH; + extern const int AMBIGUOUS_IDENTIFIER; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -1391,10 +1392,11 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) { const auto & from_join_node = table_expression_node->as(); + bool join_node_in_resolve_process = scope.table_expressions_in_resolve_process.contains(table_expression_node.get()); std::unordered_map join_using_column_name_to_column_node; - if (from_join_node.isUsingJoinExpression()) + if (!join_node_in_resolve_process && from_join_node.isUsingJoinExpression()) { auto & join_using_list = from_join_node.getJoinExpression()->as(); @@ -1411,7 +1413,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo std::optional resolved_side; QueryTreeNodePtr resolved_identifier; - auto resolve_from_using_column = [&](const QueryTreeNodePtr & using_column, JoinTableSide expression_side, Identifier display_identifier) + auto resolve_from_using_column = [&](const QueryTreeNodePtr & using_column, JoinTableSide expression_side) { auto & using_column_node = using_column->as(); auto & using_expression_list = using_column_node.getExpression()->as(); @@ -1421,7 +1423,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo auto result_column_node = inner_column_node->clone(); auto & result_column = result_column_node->as(); - result_column.setDisplayIdentifier(display_identifier); result_column.setColumnType(using_column_node.getColumnType()); return result_column_node; @@ -1439,11 +1440,11 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo && left_resolved_column.getColumnName() == right_resolved_column.getColumnName()) { JoinTableSide using_column_inner_column_table_side = isRight(join_kind) ? JoinTableSide::Right : JoinTableSide::Left; - resolved_identifier = resolve_from_using_column(using_column_node_it->second, using_column_inner_column_table_side, left_resolved_column.getDisplayIdentifier()); + resolved_identifier = resolve_from_using_column(using_column_node_it->second, using_column_inner_column_table_side); } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::AMBIGUOUS_IDENTIFIER, "JOIN {} ambigious identifier {}. In scope {}", table_expression_node->formatASTForErrorMessage(), identifier_lookup.identifier.getFullName(), @@ -1455,11 +1456,20 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo resolved_side = JoinTableSide::Left; auto & left_resolved_column = left_resolved_identifier->as(); + resolved_identifier = left_resolved_identifier; + auto using_column_node_it = join_using_column_name_to_column_node.find(left_resolved_column.getColumnName()); - if (using_column_node_it != join_using_column_name_to_column_node.end()) - resolved_identifier = resolve_from_using_column(using_column_node_it->second, JoinTableSide::Left, left_resolved_column.getDisplayIdentifier()); + if (using_column_node_it != join_using_column_name_to_column_node.end() && + !using_column_node_it->second->getColumnType()->equals(*left_resolved_column.getColumnType())) + { + auto left_resolved_column_clone = std::static_pointer_cast(left_resolved_column.clone()); + left_resolved_column_clone->setColumnType(using_column_node_it->second->getColumnType()); + resolved_identifier = std::move(left_resolved_column_clone); + } else + { resolved_identifier = left_resolved_identifier; + } } else if (right_resolved_identifier) { @@ -1467,13 +1477,19 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo auto & right_resolved_column = right_resolved_identifier->as(); auto using_column_node_it = join_using_column_name_to_column_node.find(right_resolved_column.getColumnName()); - if (using_column_node_it != join_using_column_name_to_column_node.end()) - resolved_identifier = resolve_from_using_column(using_column_node_it->second, JoinTableSide::Right, right_resolved_column.getDisplayIdentifier()); + if (using_column_node_it != join_using_column_name_to_column_node.end() && + !using_column_node_it->second->getColumnType()->equals(*right_resolved_column.getColumnType())) + { + auto right_resolved_column_clone = std::static_pointer_cast(right_resolved_column.clone()); + right_resolved_column_clone->setColumnType(using_column_node_it->second->getColumnType()); + resolved_identifier = std::move(right_resolved_column_clone); + } else + { resolved_identifier = right_resolved_identifier; + } } - bool join_node_in_resolve_process = scope.table_expressions_in_resolve_process.contains(table_expression_node.get()); if (join_node_in_resolve_process || !resolved_identifier) return resolved_identifier; @@ -2684,20 +2700,20 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes scope.alias_name_to_expression_node.erase(node_alias); } + bool node_already_cloned = false; + if (!node && allow_table_expression) { node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::TABLE}, scope).resolved_identifier; - if (node) - { - /// If table identifier is resolved as CTE clone it - bool resolved_as_cte = node->as() && node->as()->isCTE(); + /// If table identifier is resolved as CTE clone it + bool resolved_as_cte = node && node->as() && node->as()->isCTE(); - if (resolved_as_cte) - { - node = node->clone(); - node->as().setIsCTE(false); - } + if (resolved_as_cte) + { + node_already_cloned = true; + node = node->clone(); + node->as().setIsCTE(false); } } @@ -2718,6 +2734,9 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes scope.scope_node->formatASTForErrorMessage()); } + if (!node_already_cloned) + node = node->clone(); + node->setAlias(node_alias); break; } @@ -3248,14 +3267,14 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierLookup identifier_lookup {identifier_node->getIdentifier(), IdentifierLookupContext::EXPRESSION}; auto result_left_table_expression = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_node.getLeftTableExpression(), scope); if (!result_left_table_expression) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "JOIN {} using identifier {} cannot be resolved from left table expression. In scope {}", + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "JOIN {} using identifier {} cannot be resolved from left table expression. In scope {}", join_node.formatASTForErrorMessage(), identifier_full_name, scope.scope_node->formatASTForErrorMessage()); auto result_right_table_expression = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_node.getRightTableExpression(), scope); if (!result_right_table_expression) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "JOIN {} using identifier {} cannot be resolved from right table expression. In scope {}", + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "JOIN {} using identifier {} cannot be resolved from right table expression. In scope {}", join_node.formatASTForErrorMessage(), identifier_full_name, scope.scope_node->formatASTForErrorMessage()); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 848ca04480a..b48acfa3fe7 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -64,7 +64,7 @@ namespace ErrorCodes /** ClickHouse query planner. * - * TODO: JOIN support ASOF. JOIN support strictness. JOIN support constants + * TODO: JOIN support ASOF. JOIN support strictness. JOIN support constants. JOIN support ON t1.id = t1.id * TODO: JOIN drop unnecessary columns after ON, USING section * TODO: Support display names * TODO: Support RBAC. Support RBAC for ALIAS columns. @@ -1395,9 +1395,22 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, auto right_equals_expression_side = *right_equals_expression_side_optional; if (left_equals_expression_side != right_equals_expression_side) - join_clause.addKey(equals_left_child, equals_right_child); + { + const ActionsDAG::Node * left_key = equals_left_child; + const ActionsDAG::Node * right_key = equals_right_child; + + if (left_equals_expression_side == JoinTableSide::Right) + { + left_key = equals_right_child; + right_key = equals_left_child; + } + + join_clause.addKey(left_key, right_key); + } else + { join_clause.addCondition(left_equals_expression_side, join_expressions_actions_node); + } } return; @@ -1875,7 +1888,6 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, auto columns_from_joined_table = right_plan.getCurrentDataStream().header.getNamesAndTypesList(); table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); - for (auto & column_from_joined_table : columns_from_joined_table) { if (column_from_joined_table.name.starts_with("__column")) diff --git a/tests/queries/0_stateless/02371_analyzer_join_cross.sql b/tests/queries/0_stateless/02371_analyzer_join_cross.sql index b7d29049398..0c0709fb951 100644 --- a/tests/queries/0_stateless/02371_analyzer_join_cross.sql +++ b/tests/queries/0_stateless/02371_analyzer_join_cross.sql @@ -69,9 +69,9 @@ SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, te t3.id, test_table_join_3.id, t3.value, test_table_join_3.value FROM test_table_join_1 AS t1, test_table_join_2 AS t2, test_table_join_3 AS t3; -SELECT id FROM test_table_join_1, test_table_join_2; -- { serverError 36 } +SELECT id FROM test_table_join_1, test_table_join_2; -- { serverError 207 } -SELECT value FROM test_table_join_1, test_table_join_2; -- { serverError 36 } +SELECT value FROM test_table_join_1, test_table_join_2; -- { serverError 207 } DROP TABLE test_table_join_1; DROP TABLE test_table_join_2; diff --git a/tests/queries/0_stateless/02372_analyzer_join.reference b/tests/queries/0_stateless/02372_analyzer_join.reference index 17077dfd6eb..b8a658106ff 100644 --- a/tests/queries/0_stateless/02372_analyzer_join.reference +++ b/tests/queries/0_stateless/02372_analyzer_join.reference @@ -2,8 +2,8 @@ SELECT 'JOIN INNER'; JOIN INNER -SELECT 'Join without ON conditions'; -Join without ON conditions +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; 0 Join_1_Value_0 0 Join_2_Value_0 @@ -22,30 +22,14 @@ FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 SELECT '--'; -- -SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value -FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id - INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id - INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -SELECT '--'; --- -SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, -t3.id, test_table_join_3.id, t3.value, test_table_join_3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id - INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id; -0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 -1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 -SELECT id FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } -SELECT value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } -SELECT 'Join with ON conditions'; -Join with ON conditions +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +SELECT id FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 207 } +SELECT value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 207 } +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 @@ -64,84 +48,14 @@ SELECT '--'; SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' - INNER JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -SELECT 'Join only join expression use keys'; -Join only join expression use keys -SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; -Join_1_Value_0 Join_2_Value_0 -Join_1_Value_1 Join_2_Value_1 -SELECT '--'; --- -SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; -Join_1_Value_0 Join_2_Value_0 -Join_1_Value_1 Join_2_Value_1 -SELECT '--'; --- -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id -INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id; -Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 -SELECT '--'; --- -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id -INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id; -Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 -SELECT 'Join multiple clauses'; -Join multiple clauses +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -SELECT '--'; --- -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id OR t1.id = t2.id; -Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 -Join_1_Value_0 Join_2_Value_0 Join_3_Value_1 -Join_1_Value_0 Join_2_Value_0 Join_3_Value_2 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_2 -SELECT 'Join expression aliases'; -Join expression aliases +SELECT 'JOIN expression aliases'; +JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 @@ -152,56 +66,38 @@ SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 1 Join_1_Value_1 1 Join_2_Value_1 SELECT 'JOIN LEFT'; JOIN LEFT -SELECT 'Join without ON conditions'; -Join without ON conditions +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -2 Join_1_Value_3 0 +2 Join_1_Value_2 0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -2 Join_1_Value_3 0 +2 Join_1_Value_2 0 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 -2 2 Join_1_Value_3 Join_1_Value_3 0 0 +2 2 Join_1_Value_2 Join_1_Value_2 0 0 SELECT '--'; -- -SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value -FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id - LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -2 Join_1_Value_3 0 0 Join_3_Value_0 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id - LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -2 Join_1_Value_3 0 0 Join_3_Value_0 -SELECT '--'; --- -SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, -t3.id, test_table_join_3.id, t3.value, test_table_join_3.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id - LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id; -0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 -1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 -2 2 Join_1_Value_3 Join_1_Value_3 0 0 0 0 Join_3_Value_0 Join_3_Value_0 -SELECT id FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } -SELECT value FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } -SELECT 'Join with ON conditions'; -Join with ON conditions +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +Join_1_Value_2 +SELECT id FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 207 } +SELECT value FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 207 } +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 @@ -211,177 +107,74 @@ SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 0 -2 Join_1_Value_3 0 +2 Join_1_Value_2 0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 0 -2 Join_1_Value_3 0 +2 Join_1_Value_2 0 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 0 -2 Join_1_Value_3 0 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 0 0 Join_3_Value_0 -2 Join_1_Value_3 0 0 Join_3_Value_0 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 0 0 -2 Join_1_Value_3 0 0 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' -LEFT JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 0 0 -2 Join_1_Value_3 0 0 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -LEFT JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 0 0 -2 Join_1_Value_3 0 0 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' - LEFT JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 0 0 Join_3_Value_0 -2 Join_1_Value_3 0 0 Join_3_Value_0 -SELECT 'Join only join expression use keys'; -Join only join expression use keys -SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; -Join_1_Value_0 Join_2_Value_0 -Join_1_Value_1 Join_2_Value_1 -Join_1_Value_3 -SELECT '--'; --- -SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; -Join_1_Value_0 Join_2_Value_0 -Join_1_Value_1 Join_2_Value_1 -Join_1_Value_3 -SELECT '--'; --- -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id -LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id; -Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 -Join_1_Value_3 Join_3_Value_0 -SELECT '--'; --- -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id -LEFT JOIN test_table_join_3 AS t3 ON t1.id = t3.id; -Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 -Join_1_Value_3 -SELECT 'Join multiple clauses'; -Join multiple clauses +2 Join_1_Value_2 0 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -2 Join_1_Value_3 0 -SELECT '--'; --- -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -LEFT JOIN test_table_join_3 AS t3 ON t1.id = t3.id OR t1.id = t2.id; -Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 -Join_1_Value_0 Join_2_Value_0 Join_3_Value_1 -Join_1_Value_0 Join_2_Value_0 Join_3_Value_2 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_2 -Join_1_Value_3 -SELECT 'Join expression aliases'; -Join expression aliases +2 Join_1_Value_2 0 +SELECT 'JOIN expression aliases'; +JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -2 Join_1_Value_3 0 +2 Join_1_Value_2 0 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1_id = t2_id; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -2 Join_1_Value_3 0 +2 Join_1_Value_2 0 SELECT 'JOIN RIGHT'; JOIN RIGHT -SELECT 'Join without ON conditions'; -Join without ON conditions +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -0 3 Join_2_Value_2 +0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -0 3 Join_2_Value_2 +0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 -0 0 3 3 Join_2_Value_2 Join_2_Value_2 +0 0 3 3 Join_2_Value_3 Join_2_Value_3 SELECT '--'; -- -SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value -FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id - RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_2 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id - RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -0 0 4 Join_3_Value_2 -SELECT '--'; --- -SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, -t3.id, test_table_join_3.id, t3.value, test_table_join_3.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id - RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id; -0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 -1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 -0 0 0 0 4 4 Join_3_Value_2 Join_3_Value_2 -SELECT id FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } -SELECT value FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } -SELECT 'Join with ON conditions'; -Join with ON conditions +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 + Join_2_Value_3 +SELECT id FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 207 } +SELECT value FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 207 } +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 @@ -391,189 +184,78 @@ SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 0 1 Join_2_Value_1 -0 3 Join_2_Value_2 +0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 0 1 Join_2_Value_1 -0 3 Join_2_Value_2 +0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 0 1 Join_2_Value_1 -0 3 Join_2_Value_2 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 0 1 Join_3_Value_1 -0 0 4 Join_3_Value_2 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 0 1 Join_3_Value_1 -0 0 4 Join_3_Value_2 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' -RIGHT JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 1 Join_2_Value_1 0 Join_3_Value_0 -0 3 Join_2_Value_2 0 Join_3_Value_0 -0 0 1 Join_3_Value_1 -0 0 4 Join_3_Value_2 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -RIGHT JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 0 1 Join_3_Value_1 -0 0 4 Join_3_Value_2 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' - RIGHT JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -0 0 1 Join_3_Value_1 -0 0 4 Join_3_Value_2 -SELECT 'Join only join expression use keys'; -Join only join expression use keys -SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; -Join_1_Value_0 Join_2_Value_0 -Join_1_Value_1 Join_2_Value_1 - Join_2_Value_2 -SELECT '--'; --- -SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; -Join_1_Value_0 Join_2_Value_0 -Join_1_Value_1 Join_2_Value_1 - Join_2_Value_2 -SELECT '--'; --- -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id -RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id; -Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 - Join_3_Value_2 -SELECT '--'; --- -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id -RIGHT JOIN test_table_join_3 AS t3 ON t1.id = t3.id; -Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 - Join_2_Value_2 Join_3_Value_0 - Join_3_Value_2 -SELECT 'Join multiple clauses'; -Join multiple clauses +0 3 Join_2_Value_3 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -0 3 Join_2_Value_2 -SELECT '--'; --- -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -RIGHT JOIN test_table_join_3 AS t3 ON t1.id = t3.id OR t1.id = t2.id; -Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 -Join_1_Value_0 Join_2_Value_0 Join_3_Value_1 -Join_1_Value_0 Join_2_Value_0 Join_3_Value_2 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_2 - Join_2_Value_2 Join_3_Value_0 -SELECT 'Join expression aliases'; -Join expression aliases +0 3 Join_2_Value_3 +SELECT 'JOIN expression aliases'; +JOIN expression aliases SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -0 3 Join_2_Value_2 +0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1_id = t2_id; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -0 3 Join_2_Value_2 +0 3 Join_2_Value_3 SELECT 'JOIN FULL'; JOIN FULL -SELECT 'Join without ON conditions'; -Join without ON conditions +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -2 Join_1_Value_3 0 -0 3 Join_2_Value_2 +2 Join_1_Value_2 0 +0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 1 Join_2_Value_1 -2 Join_1_Value_3 0 -0 3 Join_2_Value_2 +2 Join_1_Value_2 0 +0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; 0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 -2 2 Join_1_Value_3 Join_1_Value_3 0 0 -0 0 3 3 Join_2_Value_2 Join_2_Value_2 +2 2 Join_1_Value_2 Join_1_Value_2 0 0 +0 0 3 3 Join_2_Value_3 Join_2_Value_3 SELECT '--'; -- -SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value -FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id - FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -2 Join_1_Value_3 0 0 Join_3_Value_0 -0 3 Join_2_Value_2 0 -0 0 4 Join_3_Value_2 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id - FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 -2 Join_1_Value_3 0 0 Join_3_Value_0 -0 3 Join_2_Value_2 0 -0 0 4 Join_3_Value_2 -SELECT '--'; --- -SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, -t3.id, test_table_join_3.id, t3.value, test_table_join_3.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id - FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id; -0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 -1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 -2 2 Join_1_Value_3 Join_1_Value_3 0 0 0 0 Join_3_Value_0 Join_3_Value_0 -0 0 3 3 Join_2_Value_2 Join_2_Value_2 0 0 -0 0 0 0 4 4 Join_3_Value_2 Join_3_Value_2 -SELECT id FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } -SELECT value FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } -SELECT 'Join with ON conditions'; -Join with ON conditions +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +Join_1_Value_2 + Join_2_Value_3 +SELECT id FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 207 } +SELECT value FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 207 } +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 @@ -583,39 +265,1250 @@ SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 0 -2 Join_1_Value_3 0 +2 Join_1_Value_2 0 0 1 Join_2_Value_1 -0 3 Join_2_Value_2 +0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 0 -2 Join_1_Value_3 0 +2 Join_1_Value_2 0 0 1 Join_2_Value_1 -0 3 Join_2_Value_2 +0 3 Join_2_Value_3 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 1 Join_1_Value_1 0 -2 Join_1_Value_3 0 +2 Join_1_Value_2 0 0 1 Join_2_Value_1 -0 3 Join_2_Value_2 +0 3 Join_2_Value_3 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_2 0 +0 3 Join_2_Value_3 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_2 0 +0 3 Join_2_Value_3 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_2 0 +0 3 Join_2_Value_3 +SELECT 'First JOIN INNER second JOIN INNER'; +First JOIN INNER second JOIN INNER +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 0 0 Join_3_Value_0 -2 Join_1_Value_3 0 0 Join_3_Value_0 -0 1 Join_2_Value_1 0 -0 3 Join_2_Value_2 0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +INNER JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1_id = t2_id +INNER JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT 'First JOIN INNER second JOIN LEFT'; +First JOIN INNER second JOIN LEFT +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +LEFT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1_id = t2_id +LEFT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT 'First JOIN INNER second JOIN RIGHT'; +First JOIN INNER second JOIN RIGHT +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 + Join_3_Value_4 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +0 0 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 + Join_3_Value_4 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1_id = t2_id +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT 'First JOIN INNER second JOIN FULL'; +First JOIN INNER second JOIN FULL +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 + Join_3_Value_4 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +0 0 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 + Join_3_Value_4 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +FULL JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1_id = t2_id +FULL JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT 'First JOIN LEFT second JOIN INNER'; +First JOIN LEFT second JOIN INNER +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +INNER JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1_id = t2_id +INNER JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +SELECT 'First JOIN LEFT second JOIN LEFT'; +First JOIN LEFT second JOIN LEFT +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_2 0 0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_2 0 0 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +LEFT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1_id = t2_id +LEFT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +SELECT 'First JOIN LEFT second JOIN RIGHT'; +First JOIN LEFT second JOIN RIGHT +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 + Join_3_Value_4 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +0 0 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 + Join_3_Value_4 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1_id = t2_id +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +SELECT 'First JOIN LEFT second JOIN FULL'; +First JOIN LEFT second JOIN FULL +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 LEFT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 + Join_3_Value_4 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_2 0 0 +0 0 4 Join_3_Value_4 +0 0 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_2 0 0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 + Join_3_Value_4 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +FULL JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1_id = t2_id +FULL JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +SELECT 'First JOIN RIGHT second JOIN INNER'; +First JOIN RIGHT second JOIN INNER +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +INNER JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1_id = t2_id +INNER JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +SELECT 'First JOIN RIGHT second JOIN LEFT'; +First JOIN RIGHT second JOIN LEFT +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 3 Join_2_Value_3 0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 3 Join_2_Value_3 0 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 + Join_2_Value_3 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 3 Join_2_Value_3 0 +0 1 Join_2_Value_1 0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_3 0 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 + Join_2_Value_3 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +LEFT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 3 Join_2_Value_3 0 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1_id = t2_id +LEFT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 3 Join_2_Value_3 0 +SELECT 'First JOIN RIGHT second JOIN RIGHT'; +First JOIN RIGHT second JOIN RIGHT +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 + Join_3_Value_4 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +0 0 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 + Join_3_Value_4 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1_id = t2_id +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT 'First JOIN RIGHT second JOIN FULL'; +First JOIN RIGHT second JOIN FULL +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 RIGHT JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 3 Join_2_Value_3 0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 3 Join_2_Value_3 0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 + Join_2_Value_3 + Join_3_Value_4 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 3 Join_2_Value_3 0 +0 1 Join_2_Value_1 0 +0 0 4 Join_3_Value_4 +0 0 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_3 0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 + Join_2_Value_3 + Join_3_Value_4 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +FULL JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 3 Join_2_Value_3 0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1_id = t2_id +FULL JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +0 3 Join_2_Value_3 0 +0 0 4 Join_3_Value_4 +SELECT 'First JOIN FULL second JOIN INNER'; +First JOIN FULL second JOIN INNER +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +INNER JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id +INNER JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +SELECT 'First JOIN FULL second JOIN LEFT'; +First JOIN FULL second JOIN LEFT +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 3 Join_2_Value_3 0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 3 Join_2_Value_3 0 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 +0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +LEFT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 + Join_2_Value_3 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_2 0 0 +0 3 Join_2_Value_3 0 +0 1 Join_2_Value_1 0 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_2 0 0 +0 1 Join_2_Value_1 0 +0 3 Join_2_Value_3 0 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +LEFT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 + Join_2_Value_3 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +LEFT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 3 Join_2_Value_3 0 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id +LEFT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 3 Join_2_Value_3 0 +SELECT 'First JOIN FULL second JOIN RIGHT'; +First JOIN FULL second JOIN RIGHT +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +RIGHT JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 + Join_3_Value_4 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +0 0 1 Join_3_Value_1 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 0 1 Join_3_Value_1 +0 0 4 Join_3_Value_4 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +RIGHT JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 + Join_3_Value_4 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id +RIGHT JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 0 4 Join_3_Value_4 +SELECT 'First JOIN FULL second JOIN FULL'; +First JOIN FULL second JOIN FULL +SELECT 'JOIN ON without conditions'; +JOIN ON without conditions +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 FULL JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 3 Join_2_Value_3 0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 3 Join_2_Value_3 0 +0 0 4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +2 2 Join_1_Value_2 Join_1_Value_2 0 0 0 0 Join_3_Value_0 Join_3_Value_0 +0 0 3 3 Join_2_Value_3 Join_2_Value_3 0 0 +0 0 0 0 4 4 Join_3_Value_4 Join_3_Value_4 +SELECT '--'; +-- +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +FULL JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; +Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +Join_1_Value_2 Join_3_Value_0 + Join_2_Value_3 + Join_3_Value_4 +SELECT 'JOIN ON with conditions'; +JOIN ON with conditions +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 0 +2 Join_1_Value_2 0 0 +0 3 Join_2_Value_3 0 +0 1 Join_2_Value_1 0 +0 0 4 Join_3_Value_4 0 0 1 Join_3_Value_1 -0 0 4 Join_3_Value_2 SELECT '--'; -- SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value @@ -623,118 +1516,39 @@ FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; 0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 1 Join_1_Value_1 0 0 -2 Join_1_Value_3 0 0 +2 Join_1_Value_2 0 0 0 1 Join_2_Value_1 0 -0 3 Join_2_Value_2 0 +0 3 Join_2_Value_3 0 0 0 1 Join_3_Value_1 -0 0 4 Join_3_Value_2 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' -FULL JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 0 0 -2 Join_1_Value_3 0 0 -0 1 Join_2_Value_1 0 Join_3_Value_0 -0 3 Join_2_Value_2 0 Join_3_Value_0 -0 0 1 Join_3_Value_1 -0 0 4 Join_3_Value_2 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -FULL JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 0 0 -2 Join_1_Value_3 0 0 -0 1 Join_2_Value_1 0 -0 3 Join_2_Value_2 0 -0 0 1 Join_3_Value_1 -0 0 4 Join_3_Value_2 -SELECT '--'; --- -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' - FULL JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; -0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 -1 Join_1_Value_1 0 0 Join_3_Value_0 -2 Join_1_Value_3 0 0 Join_3_Value_0 -0 1 Join_2_Value_1 0 -0 3 Join_2_Value_2 0 -0 0 1 Join_3_Value_1 -0 0 4 Join_3_Value_2 -SELECT 'Join only join expression use keys'; -Join only join expression use keys -SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; -Join_1_Value_0 Join_2_Value_0 -Join_1_Value_1 Join_2_Value_1 -Join_1_Value_3 - Join_2_Value_2 -SELECT '--'; --- -SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; -Join_1_Value_0 Join_2_Value_0 -Join_1_Value_1 Join_2_Value_1 -Join_1_Value_3 - Join_2_Value_2 -SELECT '--'; --- -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id -FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id; -Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 -Join_1_Value_3 Join_3_Value_0 - Join_2_Value_2 - Join_3_Value_2 -SELECT '--'; --- -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id -FULL JOIN test_table_join_3 AS t3 ON t1.id = t3.id; -Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 -Join_1_Value_3 - Join_2_Value_2 Join_3_Value_0 - Join_3_Value_2 -SELECT 'Join multiple clauses'; -Join multiple clauses -SELECT t1.id, t1.value, t2.id, t2.value -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; -0 Join_1_Value_0 0 Join_2_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 -2 Join_1_Value_3 0 -0 3 Join_2_Value_2 -SELECT '--'; --- +0 0 4 Join_3_Value_4 +SELECT 'JOIN multiple clauses'; +JOIN multiple clauses SELECT t1.value, t2.value, t3.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -FULL JOIN test_table_join_3 AS t3 ON t1.id = t3.id OR t1.id = t2.id; +FULL JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 -Join_1_Value_0 Join_2_Value_0 Join_3_Value_1 -Join_1_Value_0 Join_2_Value_0 Join_3_Value_2 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_0 -Join_1_Value_1 Join_2_Value_1 Join_3_Value_2 -Join_1_Value_3 - Join_2_Value_2 Join_3_Value_0 -SELECT 'Join expression aliases'; -Join expression aliases -SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); -0 Join_1_Value_0 0 Join_2_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 -2 Join_1_Value_3 0 -0 3 Join_2_Value_2 +Join_1_Value_2 Join_3_Value_0 + Join_2_Value_3 + Join_3_Value_4 +SELECT 'JOIN expression aliases'; +JOIN expression aliases +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +FULL JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 3 Join_2_Value_3 0 +0 0 4 Join_3_Value_4 SELECT '--'; -- -SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id; -0 Join_1_Value_0 0 Join_2_Value_0 -1 Join_1_Value_1 1 Join_2_Value_1 -2 Join_1_Value_3 0 -0 3 Join_2_Value_2 +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1_id = t2_id +FULL JOIN test_table_join_3 AS t3 ON t2_id = t3_id; +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +2 Join_1_Value_2 0 0 Join_3_Value_0 +0 3 Join_2_Value_3 0 +0 0 4 Join_3_Value_4 diff --git a/tests/queries/0_stateless/02372_analyzer_join.sql.j2 b/tests/queries/0_stateless/02372_analyzer_join.sql.j2 index 966c96253b5..4259bad04ba 100644 --- a/tests/queries/0_stateless/02372_analyzer_join.sql.j2 +++ b/tests/queries/0_stateless/02372_analyzer_join.sql.j2 @@ -23,15 +23,15 @@ CREATE TABLE test_table_join_3 INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value_0'); INSERT INTO test_table_join_1 VALUES (1, 'Join_1_Value_1'); -INSERT INTO test_table_join_1 VALUES (2, 'Join_1_Value_3'); +INSERT INTO test_table_join_1 VALUES (2, 'Join_1_Value_2'); INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value_0'); INSERT INTO test_table_join_2 VALUES (1, 'Join_2_Value_1'); -INSERT INTO test_table_join_2 VALUES (3, 'Join_2_Value_2'); +INSERT INTO test_table_join_2 VALUES (3, 'Join_2_Value_3'); INSERT INTO test_table_join_3 VALUES (0, 'Join_3_Value_0'); INSERT INTO test_table_join_3 VALUES (1, 'Join_3_Value_1'); -INSERT INTO test_table_join_3 VALUES (4, 'Join_3_Value_2'); +INSERT INTO test_table_join_3 VALUES (4, 'Join_3_Value_4'); -- { echoOn } @@ -39,7 +39,7 @@ INSERT INTO test_table_join_3 VALUES (4, 'Join_3_Value_2'); SELECT 'JOIN {{ join_type }}'; -SELECT 'Join without ON conditions'; +SELECT 'JOIN ON without conditions'; SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; @@ -56,28 +56,14 @@ FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1. SELECT '--'; -SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value -FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id - {{ join_type }} JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id; -SELECT '--'; +SELECT id FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 207 } -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id - {{ join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id; +SELECT value FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 207 } -SELECT '--'; - -SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, -t3.id, test_table_join_3.id, t3.value, test_table_join_3.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id - {{ join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id; - -SELECT id FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } - -SELECT value FROM test_table_join_1 {{ join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } - -SELECT 'Join with ON conditions'; +SELECT 'JOIN ON with conditions'; SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; @@ -97,73 +83,12 @@ SELECT '--'; SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' -{{ join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -{{ join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' -{{ join_type }} JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' -{{ join_type }} JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; - -SELECT '--'; - -SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' - {{ join_type }} JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; - -SELECT 'Join only join expression use keys'; - -SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id; - -SELECT '--'; - -SELECT t1.value, t2.value -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id; - -SELECT '--'; - -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -{{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id -{{ join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id; - -SELECT '--'; - -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -{{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id -{{ join_type }} JOIN test_table_join_3 AS t3 ON t1.id = t3.id; - -SELECT 'Join multiple clauses'; +SELECT 'JOIN multiple clauses'; SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id; -SELECT '--'; - -SELECT t1.value, t2.value, t3.value -FROM test_table_join_1 AS t1 -{{ join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id -{{ join_type }} JOIN test_table_join_3 AS t3 ON t1.id = t3.id OR t1.id = t2.id; - -SELECT 'Join expression aliases'; +SELECT 'JOIN expression aliases'; SELECT t1_id, t1.value, t2_id, t2.value FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id); @@ -173,6 +98,69 @@ SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value FROM test_table_join_1 {% endfor %} +{% for first_join_type in ['INNER', 'LEFT', 'RIGHT', 'FULL'] -%} +{% for second_join_type in ['INNER', 'LEFT', 'RIGHT', 'FULL'] -%} + +SELECT 'First JOIN {{ first_join_type }} second JOIN {{ second_join_type }}'; + +SELECT 'JOIN ON without conditions'; + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 {{ first_join_type }} JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +{{ second_join_type }} JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; + +SELECT '--'; +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON test_table_join_2.id = test_table_join_3.id; + +SELECT 'JOIN ON with conditions'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; + +SELECT 'JOIN multiple clauses'; + +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +{{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id OR t1.id = t2.id +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2.id = t3.id OR t3.id = t2.id; + +SELECT 'JOIN expression aliases'; + +SELECT t1_id, t1.value, t2_id, t2.value, t3_id, t3.value +FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON (t1.id AS t1_id) = (t2.id AS t2_id) +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2_id = (t3.id AS t3_id); + +SELECT '--'; + +SELECT t1.id AS t1_id, t1.value, t2.id AS t2_id, t2.value, t3.id AS t3_id, t3.value +FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1_id = t2_id +{{ second_join_type }} JOIN test_table_join_3 AS t3 ON t2_id = t3_id; + +{% endfor %} +{% endfor %} + -- { echoOff } DROP TABLE test_table_join_1; diff --git a/tests/queries/0_stateless/02373_analyzer_join_use_nulls.reference b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.reference index 20c5c1c464f..3722c23e4a0 100644 --- a/tests/queries/0_stateless/02373_analyzer_join_use_nulls.reference +++ b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.reference @@ -10,47 +10,51 @@ SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_va FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; 0 UInt64 Join_1_Value_0 String 0 Nullable(UInt64) Join_2_Value_0 Nullable(String) 1 UInt64 Join_1_Value_1 String 1 Nullable(UInt64) Join_2_Value_1 Nullable(String) -2 UInt64 Join_1_Value_3 String \N Nullable(UInt64) \N Nullable(String) +2 UInt64 Join_1_Value_2 String \N Nullable(UInt64) \N Nullable(String) SELECT '--'; -- SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id; 0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 UInt64 Join_2_Value_0 String 1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 UInt64 Join_2_Value_1 String -\N Nullable(UInt64) \N Nullable(String) 3 UInt64 Join_2_Value_2 String +\N Nullable(UInt64) \N Nullable(String) 3 UInt64 Join_2_Value_3 String SELECT '--'; -- SELECT t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; 0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 Nullable(UInt64) Join_2_Value_0 Nullable(String) 1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 Nullable(UInt64) Join_2_Value_1 Nullable(String) -2 Nullable(UInt64) Join_1_Value_3 Nullable(String) \N Nullable(UInt64) \N Nullable(String) -\N Nullable(UInt64) \N Nullable(String) 3 Nullable(UInt64) Join_2_Value_2 Nullable(String) +2 Nullable(UInt64) Join_1_Value_2 Nullable(String) \N Nullable(UInt64) \N Nullable(String) +\N Nullable(UInt64) \N Nullable(String) 3 Nullable(UInt64) Join_2_Value_3 Nullable(String) SELECT '--'; -- -SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id); 0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String SELECT '--'; -- -SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id); 0 UInt64 0 UInt64 Join_1_Value_0 String 0 Nullable(UInt64) Join_2_Value_0 Nullable(String) 1 UInt64 1 UInt64 Join_1_Value_1 String 1 Nullable(UInt64) Join_2_Value_1 Nullable(String) -2 UInt64 2 UInt64 Join_1_Value_3 String \N Nullable(UInt64) \N Nullable(String) +2 UInt64 2 UInt64 Join_1_Value_2 String \N Nullable(UInt64) \N Nullable(String) SELECT '--'; -- -SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id); 0 UInt64 0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 UInt64 Join_2_Value_0 String 1 UInt64 1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 UInt64 Join_2_Value_1 String -3 UInt64 \N Nullable(UInt64) \N Nullable(String) 3 UInt64 Join_2_Value_2 String +3 UInt64 \N Nullable(UInt64) \N Nullable(String) 3 UInt64 Join_2_Value_3 String SELECT '--'; -- -SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id); 0 Nullable(UInt64) 0 Nullable(UInt64) Join_1_Value_0 Nullable(String) 0 Nullable(UInt64) Join_2_Value_0 Nullable(String) 1 Nullable(UInt64) 1 Nullable(UInt64) Join_1_Value_1 Nullable(String) 1 Nullable(UInt64) Join_2_Value_1 Nullable(String) -2 Nullable(UInt64) 2 Nullable(UInt64) Join_1_Value_3 Nullable(String) \N Nullable(UInt64) \N Nullable(String) -\N Nullable(UInt64) \N Nullable(UInt64) \N Nullable(String) 3 Nullable(UInt64) Join_2_Value_2 Nullable(String) +2 Nullable(UInt64) 2 Nullable(UInt64) Join_1_Value_2 Nullable(String) \N Nullable(UInt64) \N Nullable(String) +\N Nullable(UInt64) \N Nullable(UInt64) \N Nullable(String) 3 Nullable(UInt64) Join_2_Value_3 Nullable(String) diff --git a/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql index 1d07589dbe9..a67a11bf8b0 100644 --- a/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql +++ b/tests/queries/0_stateless/02373_analyzer_join_use_nulls.sql @@ -17,11 +17,11 @@ CREATE TABLE test_table_join_2 INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value_0'); INSERT INTO test_table_join_1 VALUES (1, 'Join_1_Value_1'); -INSERT INTO test_table_join_1 VALUES (2, 'Join_1_Value_3'); +INSERT INTO test_table_join_1 VALUES (2, 'Join_1_Value_2'); INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value_0'); INSERT INTO test_table_join_2 VALUES (1, 'Join_2_Value_1'); -INSERT INTO test_table_join_2 VALUES (3, 'Join_2_Value_2'); +INSERT INTO test_table_join_2 VALUES (3, 'Join_2_Value_3'); -- { echoOn } @@ -45,22 +45,26 @@ FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id; SELECT '--'; -SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id); SELECT '--'; -SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id); SELECT '--'; -SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id); SELECT '--'; -SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id); -- { echoOff } diff --git a/tests/queries/0_stateless/02374_analyzer_join_using.reference b/tests/queries/0_stateless/02374_analyzer_join_using.reference new file mode 100644 index 00000000000..62750c33f89 --- /dev/null +++ b/tests/queries/0_stateless/02374_analyzer_join_using.reference @@ -0,0 +1,452 @@ +-- { echoOn } + +SELECT 'JOIN INNER'; +JOIN INNER +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id); +0 UInt16 0 UInt16 Join_1_Value_0 String 0 UInt16 Join_2_Value_0 String +1 UInt16 1 UInt16 Join_1_Value_1 String 1 UInt16 Join_2_Value_1 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id); +Join_1_Value_0 String Join_2_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id); +1 +1 +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (test_value); -- { serverError 47 } +SELECT 'JOIN LEFT'; +JOIN LEFT +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id); +0 UInt16 0 UInt16 Join_1_Value_0 String 0 UInt16 Join_2_Value_0 String +1 UInt16 1 UInt16 Join_1_Value_1 String 1 UInt16 Join_2_Value_1 String +2 UInt16 2 UInt16 Join_1_Value_2 String 0 UInt16 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id); +Join_1_Value_0 String Join_2_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String +Join_1_Value_2 String String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id); +1 +1 +1 +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (test_value); -- { serverError 47 } +SELECT 'JOIN RIGHT'; +JOIN RIGHT +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id); +0 UInt16 0 UInt16 Join_1_Value_0 String 0 UInt16 Join_2_Value_0 String +1 UInt16 1 UInt16 Join_1_Value_1 String 1 UInt16 Join_2_Value_1 String +3 UInt16 0 UInt16 String 3 UInt16 Join_2_Value_3 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id); +Join_1_Value_0 String Join_2_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String + String Join_2_Value_3 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id); +1 +1 +1 +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (test_value); -- { serverError 47 } +SELECT 'JOIN FULL'; +JOIN FULL +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id); +0 UInt16 0 UInt16 Join_1_Value_0 String 0 UInt16 Join_2_Value_0 String +1 UInt16 1 UInt16 Join_1_Value_1 String 1 UInt16 Join_2_Value_1 String +2 UInt16 2 UInt16 Join_1_Value_2 String 0 UInt16 String +0 UInt16 0 UInt16 String 3 UInt16 Join_2_Value_3 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id); +Join_1_Value_0 String Join_2_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String +Join_1_Value_2 String String + String Join_2_Value_3 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id); +1 +1 +1 +1 +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (test_value); -- { serverError 47 } +SELECT 'First JOIN INNER second JOIN INNER'; +First JOIN INNER second JOIN INNER +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +1 +1 +SELECT id FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN INNER second JOIN LEFT'; +First JOIN INNER second JOIN LEFT +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +1 +1 +SELECT id FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id LEFT JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN INNER second JOIN RIGHT'; +First JOIN INNER second JOIN RIGHT +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +4 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String + String String Join_3_Value_4 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +1 +1 +1 +SELECT id FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id RIGHT JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN INNER second JOIN FULL'; +First JOIN INNER second JOIN FULL +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +0 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String + String String Join_3_Value_4 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +1 +1 +1 +SELECT id FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id FULL JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN LEFT second JOIN INNER'; +First JOIN LEFT second JOIN INNER +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +1 +1 +SELECT id FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN LEFT second JOIN LEFT'; +First JOIN LEFT second JOIN LEFT +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +2 UInt64 2 UInt64 Join_1_Value_2 String 0 UInt64 String 0 UInt64 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String +Join_1_Value_2 String String String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +1 +1 +1 +SELECT id FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id LEFT JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN LEFT second JOIN RIGHT'; +First JOIN LEFT second JOIN RIGHT +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +4 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String + String String Join_3_Value_4 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +1 +1 +1 +SELECT id FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id RIGHT JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN LEFT second JOIN FULL'; +First JOIN LEFT second JOIN FULL +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +2 UInt64 2 UInt64 Join_1_Value_2 String 0 UInt64 String 0 UInt64 String +0 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String +Join_1_Value_2 String String String + String String Join_3_Value_4 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +1 +1 +1 +1 +SELECT id FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id FULL JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN RIGHT second JOIN INNER'; +First JOIN RIGHT second JOIN INNER +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +1 +1 +SELECT id FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN RIGHT second JOIN LEFT'; +First JOIN RIGHT second JOIN LEFT +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +3 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String + String Join_2_Value_3 String String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +1 +1 +1 +SELECT id FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id LEFT JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN RIGHT second JOIN RIGHT'; +First JOIN RIGHT second JOIN RIGHT +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +4 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String + String String Join_3_Value_4 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +1 +1 +1 +SELECT id FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id RIGHT JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN RIGHT second JOIN FULL'; +First JOIN RIGHT second JOIN FULL +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +3 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 String +0 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String + String Join_2_Value_3 String String + String String Join_3_Value_4 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +1 +1 +1 +1 +SELECT id FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id FULL JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN FULL second JOIN INNER'; +First JOIN FULL second JOIN INNER +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +0 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 Join_3_Value_0 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String + String Join_2_Value_3 String Join_3_Value_0 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +1 +1 +1 +SELECT id FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN FULL second JOIN LEFT'; +First JOIN FULL second JOIN LEFT +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +2 UInt64 2 UInt64 Join_1_Value_2 String 0 UInt64 String 0 UInt64 String +0 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 Join_3_Value_0 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String +Join_1_Value_2 String String String + String Join_2_Value_3 String Join_3_Value_0 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +1 +1 +1 +1 +SELECT id FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id LEFT JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN FULL second JOIN RIGHT'; +First JOIN FULL second JOIN RIGHT +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +0 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 Join_3_Value_0 String +4 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String + String Join_2_Value_3 String Join_3_Value_0 String + String String Join_3_Value_4 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +1 +1 +1 +1 +SELECT id FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id RIGHT JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } +SELECT 'First JOIN FULL second JOIN FULL'; +First JOIN FULL second JOIN FULL +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +0 UInt64 0 UInt64 Join_1_Value_0 String 0 UInt64 Join_2_Value_0 String 0 UInt64 Join_3_Value_0 String +1 UInt64 1 UInt64 Join_1_Value_1 String 1 UInt64 Join_2_Value_1 String 1 UInt64 Join_3_Value_1 String +2 UInt64 2 UInt64 Join_1_Value_2 String 0 UInt64 String 0 UInt64 String +0 UInt64 0 UInt64 String 3 UInt64 Join_2_Value_3 String 0 UInt64 Join_3_Value_0 String +0 UInt64 0 UInt64 String 0 UInt64 String 4 UInt64 Join_3_Value_4 String +SELECT '--'; +-- +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String +Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String +Join_1_Value_2 String String String + String Join_2_Value_3 String Join_3_Value_0 String + String String Join_3_Value_4 String +SELECT '--'; +-- +SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +1 +1 +1 +1 +1 +SELECT id FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id FULL JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } diff --git a/tests/queries/0_stateless/02374_analyzer_join_using.sql.j2 b/tests/queries/0_stateless/02374_analyzer_join_using.sql.j2 new file mode 100644 index 00000000000..a8e9cc9eefc --- /dev/null +++ b/tests/queries/0_stateless/02374_analyzer_join_using.sql.j2 @@ -0,0 +1,87 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table_join_1; +CREATE TABLE test_table_join_1 +( + id UInt8, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_2; +CREATE TABLE test_table_join_2 +( + id UInt16, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_3; +CREATE TABLE test_table_join_3 +( + id UInt64, + value String +) ENGINE = TinyLog; + +INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value_0'); +INSERT INTO test_table_join_1 VALUES (1, 'Join_1_Value_1'); +INSERT INTO test_table_join_1 VALUES (2, 'Join_1_Value_2'); + +INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value_0'); +INSERT INTO test_table_join_2 VALUES (1, 'Join_2_Value_1'); +INSERT INTO test_table_join_2 VALUES (3, 'Join_2_Value_3'); + +INSERT INTO test_table_join_3 VALUES (0, 'Join_3_Value_0'); +INSERT INTO test_table_join_3 VALUES (1, 'Join_3_Value_1'); +INSERT INTO test_table_join_3 VALUES (4, 'Join_3_Value_4'); + +-- { echoOn } + +{% for join_type in ['INNER', 'LEFT', 'RIGHT', 'FULL'] -%} + +SELECT 'JOIN {{ join_type }}'; + +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 USING (id); + +SELECT '--'; + +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 USING (id); + +SELECT '--'; + +SELECT 1 FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 USING (id); + +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 USING (test_value); -- { serverError 47 } + +{% endfor %} + +{% for first_join_type in ['INNER', 'LEFT', 'RIGHT', 'FULL'] -%} +{% for second_join_type in ['INNER', 'LEFT', 'RIGHT', 'FULL'] -%} + +SELECT 'First JOIN {{ first_join_type }} second JOIN {{ second_join_type }}'; + +SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), +t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value), t3.id AS t3_id, toTypeName(t3_id), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 USING (id) {{ second_join_type }} JOIN test_table_join_3 AS t3 USING(id); + +SELECT '--'; + +SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value), t3.value AS t3_value, toTypeName(t3_value) +FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 USING (id) {{ second_join_type }} JOIN test_table_join_3 AS t3 USING(id); + +SELECT '--'; + +SELECT 1 FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 USING (id) {{ second_join_type }} JOIN test_table_join_3 AS t3 USING(id); + +SELECT id FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id {{ second_join_type }} JOIN test_table_join_3 AS t3 USING (id); -- { serverError 207 } + +{% endfor %} +{% endfor %} + +-- { echoOff } + +DROP TABLE test_table_join_1; +DROP TABLE test_table_join_2; +DROP TABLE test_table_join_3; From 0da001963431d7862bb358b3340fc84936e6abf9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 21 Aug 2022 13:46:07 +0200 Subject: [PATCH 070/188] Added ARRAY JOIN support --- src/Analyzer/ArrayJoinNode.cpp | 3 +- src/Analyzer/QueryAnalysisPass.cpp | 93 ++++++++------- src/Analyzer/QueryTreeBuilder.cpp | 5 +- .../InterpreterSelectQueryAnalyzer.cpp | 70 +++++++++-- .../02374_analyzer_array_join.reference | 110 ++++++++++++++++++ .../0_stateless/02374_analyzer_array_join.sql | 70 +++++++++++ 6 files changed, 294 insertions(+), 57 deletions(-) create mode 100644 tests/queries/0_stateless/02374_analyzer_array_join.reference create mode 100644 tests/queries/0_stateless/02374_analyzer_array_join.sql diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp index c85f52b1657..074acc2e8af 100644 --- a/src/Analyzer/ArrayJoinNode.cpp +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -32,7 +32,7 @@ void ArrayJoinNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_stat buffer << '\n' << std::string(indent + 2, ' ') << "TABLE EXPRESSION\n"; getTableExpression()->dumpTreeImpl(buffer, format_state, indent + 4); - buffer << '\n' << std::string(indent + 2, ' ') << "JOIN EXPRESSSION\n"; + buffer << '\n' << std::string(indent + 2, ' ') << "JOIN EXPRESSSIONS\n"; getJoinExpressionsNode()->dumpTreeImpl(buffer, format_state, indent + 4); } @@ -51,6 +51,7 @@ ASTPtr ArrayJoinNode::toASTImpl() const { auto array_join_ast = std::make_shared(); array_join_ast->kind = is_left ? ASTArrayJoin::Kind::Left : ASTArrayJoin::Kind::Inner; + const auto & join_expression_list_node = getJoinExpressionsNode(); array_join_ast->children.push_back(join_expression_list_node->toAST()); array_join_ast->expression_list = array_join_ast->children.back(); diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index d33ed683799..eb47ab73191 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -784,6 +784,8 @@ private: QueryTreeNodePtr tryResolveIdentifierFromJoin(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + QueryTreeNodePtr tryResolveIdentifierFromArrayJoin(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + QueryTreeNodePtr tryResolveIdentifierFromJoinTreeNode(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope); QueryTreeNodePtr tryResolveIdentifierFromJoinTree(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); @@ -1508,6 +1510,44 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo return resolved_identifier; } +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromArrayJoin(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) +{ + const auto & from_array_join_node = table_expression_node->as(); + auto resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_array_join_node.getTableExpression(), scope); + + /** Special case when qualified or unqualified identifier point to array join expression without alias. + * + * CREATE TABLE test_table (id UInt64, value String, value_array Array(UInt8)) ENGINE=TinyLog; + * SELECT id, value, value_array, test_table.value_array, default.test_table.value_array FROM test_table ARRAY JOIN value_array; + * + * value_array, test_table.value_array, default.test_table.value_array must be resolved into array join expression. + */ + if (!scope.table_expressions_in_resolve_process.contains(table_expression_node.get()) && resolved_identifier) + { + auto & resolved_identifier_column = resolved_identifier->as(); + + for (const auto & array_join_expression : from_array_join_node.getJoinExpressions().getNodes()) + { + auto & array_join_column_expression = array_join_expression->as(); + if (array_join_column_expression.hasAlias()) + continue; + + auto & array_join_column_inner_expression = array_join_column_expression.getExpressionOrThrow(); + if (array_join_column_inner_expression.get() == resolved_identifier.get() || + array_join_column_inner_expression->isEqual(*resolved_identifier)) + { + auto array_join_column = array_join_column_expression.getColumn(); + auto result = std::make_shared(array_join_column, table_expression_node); + result->setDisplayIdentifier(resolved_identifier_column.getDisplayIdentifier()); + + return result; + } + } + } + + return resolved_identifier; +} + QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTreeNode(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope) { auto join_tree_node_type = join_tree_node->getNodeType(); @@ -1520,37 +1560,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTreeNode(const Ident } case QueryTreeNodeType::ARRAY_JOIN: { - const auto & from_array_join_node = join_tree_node->as(); - auto resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_array_join_node.getTableExpression(), scope); - - /** Special case when qualified or unqualified identifier point to array join expression. - * - * CREATE TABLE test_table (id UInt64, value String, value_array Array(UInt8)) ENGINE=TinyLog; - * SELECT id, value, value_array, test_table.value_array, default.test_table.value_array FROM test_table ARRAY JOIN value_array; - * - * value_array, test_table.value_array, default.test_table.value_array must be resolved into array join expression. - */ - if (resolved_identifier) - { - for (const auto & array_join_expression : from_array_join_node.getJoinExpressions().getNodes()) - { - auto * array_join_column_expression = array_join_expression->as(); - if (!array_join_column_expression) - continue; - - auto & array_join_column_inner_expression = array_join_column_expression->getExpression(); - - if (!array_join_column_inner_expression) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "ARRAY JOIN column node must have inner expression. In scope {}", - scope.scope_node->formatASTForErrorMessage()); - - if (array_join_expression.get() == resolved_identifier.get() || array_join_expression->isEqual(*resolved_identifier)) - return array_join_expression; - } - } - - return resolved_identifier; + return tryResolveIdentifierFromArrayJoin(identifier_lookup, join_tree_node, scope); } case QueryTreeNodeType::QUERY: [[fallthrough]]; @@ -3199,8 +3209,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, auto array_join_expression_alias = array_join_expression->getAlias(); if (!array_join_expression_alias.empty() && scope.alias_name_to_expression_node.contains(array_join_expression_alias)) - throw Exception( - ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, + throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, "ARRAY JOIN expression {} with duplicate alias {}. In scope {}", array_join_expression->formatASTForErrorMessage(), array_join_expression_alias, @@ -3217,16 +3226,12 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) { auto array_join_expression_alias = array_join_expression->getAlias(); - - auto result_name = "__array_join_expression_" + std::to_string(array_join_expressions_counter); - ++array_join_expressions_counter; - auto result_type = array_join_expression->getResultType(); if (!isArray(result_type)) - throw Exception( - ErrorCodes::TYPE_MISMATCH, - "ARRAY JOIN requires expression with Array type. Actual {}. In scope {}", + throw Exception(ErrorCodes::TYPE_MISMATCH, + "ARRAY JOIN {} requires expression with Array type. Actual {}. In scope {}", + array_join_node.formatASTForErrorMessage(), result_type->getName(), scope.scope_node->formatASTForErrorMessage()); @@ -3234,14 +3239,16 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, result_type = assert_cast(*result_type).getNestedType(); - auto array_join_column = std::make_shared( - NameAndTypePair{result_name, result_type}, array_join_expression, join_tree_node); + auto array_join_expression_name = "__array_join_expression_" + std::to_string(array_join_expressions_counter); + ++array_join_expressions_counter; + + auto array_join_column = std::make_shared(NameAndTypePair{array_join_expression_name, result_type}, array_join_expression, join_tree_node); array_join_expression = std::move(array_join_column); array_join_expression->setAlias(array_join_expression_alias); auto it = scope.alias_name_to_expression_node.find(array_join_expression_alias); if (it != scope.alias_name_to_expression_node.end()) - it->second = array_join_expression; + it->second = std::make_shared(NameAndTypePair{array_join_expression_name, result_type}, join_tree_node); } break; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index cf6f541cbfb..fa4f5b18c83 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -459,10 +459,11 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q table_expressions.pop_back(); auto array_join_expressions_list = getExpressionList(array_join_expression.expression_list); - auto array_join_node = std::make_shared(std::move(last_table_expression), std::move(array_join_expressions_list), is_left_array_join); - array_join_node->setOriginalAST(table_element.array_join); + /** Original AST is not set because it will contain only array join part and does + * not include left table expression. + */ table_expressions.push_back(std::move(array_join_node)); } } diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index b48acfa3fe7..53b745babb9 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -42,13 +42,14 @@ #include #include #include +#include #include #include #include #include #include - +#include namespace DB { @@ -74,7 +75,6 @@ namespace ErrorCodes * TODO: Support ORDER BY, LIMIT * TODO: Support WINDOW FUNCTIONS * TODO: Support DISTINCT - * TODO: Support ArrayJoin * TODO: Support building sets for IN functions * TODO: Support trivial count optimization * TODO: Support totals, extremes @@ -994,7 +994,12 @@ public: } case QueryTreeNodeType::ARRAY_JOIN: { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ARRAY JOIN is not supported"); + auto & array_join_node = join_tree_node->as(); + visit(array_join_node.getTableExpression(), planner_context); + + std::string table_expression_identifier = std::to_string(planner_context.table_expression_node_to_identifier.size()); + planner_context.table_expression_node_to_identifier.emplace(join_tree_node.get(), table_expression_identifier); + break; } default: { @@ -1025,10 +1030,8 @@ public: auto column_source_node = column_node->getColumnSource(); auto column_source_node_type = column_source_node->getNodeType(); - if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ARRAY JOIN is not supported"); - - if (column_source_node_type == QueryTreeNodeType::LAMBDA) + if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN || + column_source_node_type == QueryTreeNodeType::LAMBDA) return; /// JOIN using expression @@ -1682,11 +1685,11 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, planner_context); auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); - left_join_expressions_actions_step->setStepDescription("Join actions"); + left_join_expressions_actions_step->setStepDescription("JOIN actions"); left_plan.addStep(std::move(left_join_expressions_actions_step)); auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); - right_join_expressions_actions_step->setStepDescription("Join actions"); + right_join_expressions_actions_step->setStepDescription("JOIN actions"); right_plan.addStep(std::move(right_join_expressions_actions_step)); } @@ -1918,6 +1921,49 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, return result_plan; } +QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContext & planner_context) +{ + auto & array_join_node = table_expression->as(); + + auto left_plan = buildQueryPlanForJoinTreeNode(array_join_node.getTableExpression(), + select_query_info, + select_query_options, + planner_context); + auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + ActionsDAGPtr array_join_action_dag = std::make_shared(left_plan_output_columns); + QueryTreeActionsVisitor actions_visitor(array_join_action_dag, planner_context); + + NameSet array_join_columns; + for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) + { + auto & array_join_expression_column = array_join_expression->as(); + const auto & array_join_column_name = array_join_expression_column.getColumnName(); + array_join_columns.insert(array_join_column_name); + + auto expression_dag_index_nodes = actions_visitor.visit(array_join_expression_column.getExpressionOrThrow()); + for (auto & expression_dag_index_node : expression_dag_index_nodes) + { + const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_name); + array_join_action_dag->getOutputs().push_back(array_join_column_node); + } + } + + auto array_join_actions = std::make_unique(left_plan.getCurrentDataStream(), array_join_action_dag); + array_join_actions->setStepDescription("ARRAY JOIN actions"); + left_plan.addStep(std::move(array_join_actions)); + + auto array_join_action = std::make_shared(array_join_columns, array_join_node.isLeft(), planner_context.query_context); + auto array_join_step = std::make_unique(left_plan.getCurrentDataStream(), std::move(array_join_action)); + array_join_step->setStepDescription("ARRAY JOIN"); + left_plan.addStep(std::move(array_join_step)); + + return left_plan; +} + QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, @@ -1942,11 +1988,13 @@ QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, } case QueryTreeNodeType::ARRAY_JOIN: { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ARRAY JOIN is not supported"); + return buildQueryPlanForArrayJoinNode(join_tree_node, select_query_info, select_query_options, planner_context); } default: { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected query, table, table function, join or array join query node. Actual {}", join_tree_node->formatASTForErrorMessage()); + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected query, table, table function, join or array join query node. Actual {}", + join_tree_node->formatASTForErrorMessage()); } } } diff --git a/tests/queries/0_stateless/02374_analyzer_array_join.reference b/tests/queries/0_stateless/02374_analyzer_array_join.reference new file mode 100644 index 00000000000..28859f715b3 --- /dev/null +++ b/tests/queries/0_stateless/02374_analyzer_array_join.reference @@ -0,0 +1,110 @@ +-- { echoOn } + +SELECT 'ARRAY JOIN with constant'; +ARRAY JOIN with constant +SELECT id, value, value_1 FROM test_table ARRAY JOIN [1, 2, 3] AS value_1; +0 Value 1 +0 Value 2 +0 Value 3 +0 Value 1 +0 Value 2 +0 Value 3 +SELECT '--'; +-- +SELECT id, value FROM test_table ARRAY JOIN [1, 2, 3] AS value; +0 1 +0 2 +0 3 +0 1 +0 2 +0 3 +SELECT '--'; +-- +WITH [1, 2, 3] AS constant_array SELECT id, value FROM test_table ARRAY JOIN constant_array AS value; +0 1 +0 2 +0 3 +0 1 +0 2 +0 3 +SELECT '--'; +-- +WITH [1, 2, 3] AS constant_array SELECT id, value, value_1 FROM test_table ARRAY JOIN constant_array AS value_1; +0 Value 1 +0 Value 2 +0 Value 3 +0 Value 1 +0 Value 2 +0 Value 3 +SELECT '--'; +-- +SELECT id, value, value_1, value_2 FROM test_table ARRAY JOIN [[1, 2, 3]] AS value_1 ARRAY JOIN value_1 AS value_2; +0 Value [1,2,3] 1 +0 Value [1,2,3] 2 +0 Value [1,2,3] 3 +0 Value [1,2,3] 1 +0 Value [1,2,3] 2 +0 Value [1,2,3] 3 +SELECT 1 AS value FROM test_table ARRAY JOIN [1,2,3] AS value; -- { serverError 179 } +SELECT 'ARRAY JOIN with column'; +ARRAY JOIN with column +SELECT id, value, test_table.value_array FROM test_table ARRAY JOIN value_array; +0 Value 1 +0 Value 2 +0 Value 3 +0 Value 4 +0 Value 5 +0 Value 6 +SELECT '--'; +-- +SELECT id, value_array, value FROM test_table ARRAY JOIN value_array AS value; +0 [1,2,3] 1 +0 [1,2,3] 2 +0 [1,2,3] 3 +0 [4,5,6] 4 +0 [4,5,6] 5 +0 [4,5,6] 6 +SELECT '--'; +-- +SELECT id, value, value_array, value_array_element FROM test_table ARRAY JOIN value_array AS value_array_element; +0 Value [1,2,3] 1 +0 Value [1,2,3] 2 +0 Value [1,2,3] 3 +0 Value [4,5,6] 4 +0 Value [4,5,6] 5 +0 Value [4,5,6] 6 +SELECT '--'; +-- +SELECT id, value, value_array AS value_array_array_alias FROM test_table ARRAY JOIN value_array_array_alias; +0 Value [1,2,3] +0 Value [1,2,3] +0 Value [1,2,3] +0 Value [4,5,6] +0 Value [4,5,6] +0 Value [4,5,6] +SELECT '--'; +-- +SELECT id AS value FROM test_table ARRAY JOIN value_array AS value; -- { serverError 179 } +SELECT '--'; +-- +SELECT id, value, value_array AS value_array_array_alias, value_array_array_alias_element FROM test_table ARRAY JOIN value_array_array_alias AS value_array_array_alias_element; +0 Value [1,2,3] 1 +0 Value [1,2,3] 2 +0 Value [1,2,3] 3 +0 Value [4,5,6] 4 +0 Value [4,5,6] 5 +0 Value [4,5,6] 6 +SELECT '--'; +-- +SELECT id, value, value_array_array, value_array_array_inner_element, value_array_array_inner_element, value_array_array_inner_inner_element +FROM test_table ARRAY JOIN value_array_array AS value_array_array_inner_element +ARRAY JOIN value_array_array_inner_element AS value_array_array_inner_inner_element; +0 Value [[1,2,3]] [1,2,3] [1,2,3] 1 +0 Value [[1,2,3]] [1,2,3] [1,2,3] 2 +0 Value [[1,2,3]] [1,2,3] [1,2,3] 3 +0 Value [[1,2,3],[4,5,6]] [1,2,3] [1,2,3] 1 +0 Value [[1,2,3],[4,5,6]] [1,2,3] [1,2,3] 2 +0 Value [[1,2,3],[4,5,6]] [1,2,3] [1,2,3] 3 +0 Value [[1,2,3],[4,5,6]] [4,5,6] [4,5,6] 4 +0 Value [[1,2,3],[4,5,6]] [4,5,6] [4,5,6] 5 +0 Value [[1,2,3],[4,5,6]] [4,5,6] [4,5,6] 6 diff --git a/tests/queries/0_stateless/02374_analyzer_array_join.sql b/tests/queries/0_stateless/02374_analyzer_array_join.sql new file mode 100644 index 00000000000..b7e19bceadf --- /dev/null +++ b/tests/queries/0_stateless/02374_analyzer_array_join.sql @@ -0,0 +1,70 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String, + value_array Array(UInt64), + value_array_array Array(Array(UInt64)) +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value', [1, 2, 3], [[1, 2, 3]]), (0, 'Value', [4, 5, 6], [[1, 2, 3], [4, 5, 6]]); + +-- { echoOn } + +SELECT 'ARRAY JOIN with constant'; + +SELECT id, value, value_1 FROM test_table ARRAY JOIN [1, 2, 3] AS value_1; + +SELECT '--'; + +SELECT id, value FROM test_table ARRAY JOIN [1, 2, 3] AS value; + +SELECT '--'; + +WITH [1, 2, 3] AS constant_array SELECT id, value FROM test_table ARRAY JOIN constant_array AS value; + +SELECT '--'; + +WITH [1, 2, 3] AS constant_array SELECT id, value, value_1 FROM test_table ARRAY JOIN constant_array AS value_1; + +SELECT '--'; + +SELECT id, value, value_1, value_2 FROM test_table ARRAY JOIN [[1, 2, 3]] AS value_1 ARRAY JOIN value_1 AS value_2; + +SELECT 1 AS value FROM test_table ARRAY JOIN [1,2,3] AS value; -- { serverError 179 } + +SELECT 'ARRAY JOIN with column'; + +SELECT id, value, test_table.value_array FROM test_table ARRAY JOIN value_array; + +SELECT '--'; + +SELECT id, value_array, value FROM test_table ARRAY JOIN value_array AS value; + +SELECT '--'; + +SELECT id, value, value_array, value_array_element FROM test_table ARRAY JOIN value_array AS value_array_element; + +SELECT '--'; + +SELECT id, value, value_array AS value_array_array_alias FROM test_table ARRAY JOIN value_array_array_alias; + +SELECT '--'; + +SELECT id AS value FROM test_table ARRAY JOIN value_array AS value; -- { serverError 179 } + +SELECT '--'; + +SELECT id, value, value_array AS value_array_array_alias, value_array_array_alias_element FROM test_table ARRAY JOIN value_array_array_alias AS value_array_array_alias_element; + +SELECT '--'; + +SELECT id, value, value_array_array, value_array_array_inner_element, value_array_array_inner_element, value_array_array_inner_inner_element +FROM test_table ARRAY JOIN value_array_array AS value_array_array_inner_element +ARRAY JOIN value_array_array_inner_element AS value_array_array_inner_inner_element; + +-- { echoOff } + +DROP TABLE test_table; From 85193ede423c8a3dccc9f80434f5bef459a9a556 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 23 Aug 2022 11:50:02 +0200 Subject: [PATCH 071/188] Added UNION support --- src/Analyzer/IQueryTreeNode.cpp | 1 + src/Analyzer/IQueryTreeNode.h | 3 +- src/Analyzer/QueryAnalysisPass.cpp | 135 ++++++++---- src/Analyzer/QueryNode.cpp | 12 +- src/Analyzer/QueryNode.h | 10 +- src/Analyzer/QueryTreeBuilder.cpp | 182 +++++++++++------ src/Analyzer/UnionNode.cpp | 193 ++++++++++++++++++ src/Analyzer/UnionNode.h | 126 ++++++++++++ src/Analyzer/Utils.cpp | 6 - src/Analyzer/Utils.h | 3 - .../InterpreterSelectQueryAnalyzer.cpp | 174 +++++++++++++--- src/Parsers/ParserExplainQuery.cpp | 2 +- .../QueryPlan/IntersectOrExceptStep.h | 4 +- .../02375_analyzer_union.reference | 62 ++++++ .../0_stateless/02375_analyzer_union.sql | 71 +++++++ 15 files changed, 825 insertions(+), 159 deletions(-) create mode 100644 src/Analyzer/UnionNode.cpp create mode 100644 src/Analyzer/UnionNode.h create mode 100644 tests/queries/0_stateless/02375_analyzer_union.reference create mode 100644 tests/queries/0_stateless/02375_analyzer_union.sql diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 051976db228..3135ff7f96a 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -35,6 +35,7 @@ const char * toString(QueryTreeNodeType type) case QueryTreeNodeType::QUERY: return "QUERY"; case QueryTreeNodeType::ARRAY_JOIN: return "ARRAY_JOIN"; case QueryTreeNodeType::JOIN: return "JOIN"; + case QueryTreeNodeType::UNION: return "UNION"; } } diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index b45ec003bdc..2343716e82b 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -39,7 +39,8 @@ enum class QueryTreeNodeType TABLE_FUNCTION, QUERY, ARRAY_JOIN, - JOIN + JOIN, + UNION }; /// Convert query tree node type to string diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index eb47ab73191..46fa43b3be7 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -45,6 +45,7 @@ #include #include #include +#include #include #include @@ -174,7 +175,6 @@ namespace ErrorCodes * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. - * TODO: UNION * TODO: JOIN support SELF JOIN with MergeTree. JOIN support matchers. * TODO: WINDOW functions * TODO: Table expression modifiers final, sample_size, sample_offset @@ -736,19 +736,25 @@ public: { IdentifierResolveScope scope(node, nullptr /*parent_scope*/); - if (node->getNodeType() == QueryTreeNodeType::QUERY) + auto node_type = node->getNodeType(); + + if (node_type == QueryTreeNodeType::UNION) + { + resolveUnion(node, scope); + } + else if (node_type == QueryTreeNodeType::QUERY) { resolveQuery(node, scope); } - else if (node->getNodeType() == QueryTreeNodeType::LIST) + else if (node_type == QueryTreeNodeType::LIST) { resolveExpressionNodeList(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } - else if (node->getNodeType() == QueryTreeNodeType::FUNCTION) + else if (node_type == QueryTreeNodeType::FUNCTION) { resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } - else if (node->getNodeType() == QueryTreeNodeType::LAMBDA) + else if (node_type == QueryTreeNodeType::LAMBDA) { resolveLambda(node, {}, scope); } @@ -814,6 +820,8 @@ private: void resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope); + void resolveUnion(const QueryTreeNodePtr & union_node, IdentifierResolveScope & scope); + /// Query analyzer context ContextPtr context; @@ -1250,11 +1258,12 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo { auto * table_node = table_expression_node->as(); auto * query_node = table_expression_node->as(); + auto * union_node = table_expression_node->as(); auto * table_function_node = table_expression_node->as(); - if (!table_node && !table_function_node && !query_node) + if (!table_node && !table_function_node && !query_node && !union_node) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Unexpected table expression. Expected table, table function or query node. Actual {}. In scope {}", + "Unexpected table expression. Expected table, table function, query or union node. Actual {}. In scope {}", table_expression_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); @@ -1273,12 +1282,12 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo database_name = table_storage_id.database_name; table_expression_name = table_storage_id.getFullNameNotQuoted(); } - else if (query_node) + else if (query_node || union_node) { - table_name = query_node->getCTEName(); + table_name = query_node ? query_node->getCTEName() : union_node->getCTEName(); - if (query_node->hasAlias()) - table_expression_name = query_node->getAlias(); + if (table_expression_node->hasAlias()) + table_expression_name = table_expression_node->getAlias(); } else if (table_function_node) { @@ -1555,15 +1564,13 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTreeNode(const Ident switch (join_tree_node_type) { case QueryTreeNodeType::JOIN: - { return tryResolveIdentifierFromJoin(identifier_lookup, join_tree_node, scope); - } case QueryTreeNodeType::ARRAY_JOIN: - { return tryResolveIdentifierFromArrayJoin(identifier_lookup, join_tree_node, scope); - } case QueryTreeNodeType::QUERY: [[fallthrough]]; + case QueryTreeNodeType::UNION: + [[fallthrough]]; case QueryTreeNodeType::TABLE: [[fallthrough]]; case QueryTreeNodeType::TABLE_FUNCTION: @@ -1580,7 +1587,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTreeNode(const Ident default: { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Scope FROM section expected table, table function, query, join or array join. Actual {}. In scope {}", + "Scope FROM section expected table, table function, query, union, join or array join. Actual {}. In scope {}", join_tree_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } @@ -1609,10 +1616,10 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTree(const Identifie return resolved_identifier; auto * query_scope_node = scope.scope_node->as(); - if (!query_scope_node || !query_scope_node->getFrom()) + if (!query_scope_node || !query_scope_node->getJoinTree()) return {}; - const auto & join_tree_node = query_scope_node->getFrom(); + const auto & join_tree_node = query_scope_node->getJoinTree(); return tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_tree_node, scope); } @@ -1949,7 +1956,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, } /// If there are no parent scope that has tables or query scope does not have FROM section - if (!scope_query_node || !scope_query_node->getFrom()) + if (!scope_query_node || !scope_query_node->getJoinTree()) { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Unqualified matcher {} cannot be resolved. There are no table sources. In scope {}", @@ -1959,9 +1966,9 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, NamesAndTypesList initial_matcher_columns; - auto * from_query_node = scope_query_node->getFrom()->as(); - auto * from_table_node = scope_query_node->getFrom()->as(); - auto * from_table_function_node = scope_query_node->getFrom()->as(); + auto * from_query_node = scope_query_node->getJoinTree()->as(); + auto * from_table_node = scope_query_node->getJoinTree()->as(); + auto * from_table_function_node = scope_query_node->getJoinTree()->as(); if (from_query_node) { @@ -1996,14 +2003,14 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unqualified matcher resolve unexpected FROM section {}", - scope_query_node->getFrom()->formatASTForErrorMessage()); + scope_query_node->getJoinTree()->formatASTForErrorMessage()); } for (auto & column : initial_matcher_columns) { const auto & column_name = column.name; if (matcher_node_typed.isMatchingColumn(column_name)) - matcher_expression_nodes.push_back(std::make_shared(column, scope_query_node->getFrom())); + matcher_expression_nodes.push_back(std::make_shared(column, scope_query_node->getJoinTree())); } } @@ -2828,6 +2835,16 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes break; } + case QueryTreeNodeType::UNION: + { + IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); + resolveUnion(node, subquery_scope); + + if (!allow_table_expression) + evaluateScalarSubquery(node); + + break; + } case QueryTreeNodeType::ARRAY_JOIN: { throw Exception(ErrorCodes::LOGICAL_ERROR, @@ -2971,6 +2988,11 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); break; } + case QueryTreeNodeType::UNION: + { + scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); + break; + } case QueryTreeNodeType::TABLE_FUNCTION: { scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); @@ -2999,7 +3021,7 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod default: { throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Query FROM section expected table, table function, query, ARRAY JOIN or JOIN. Actual {} {}. In scope {}", + "Query FROM section expected table, table function, query, UNION, ARRAY JOIN or JOIN. Actual {} {}. In scope {}", current_join_tree_node->getNodeTypeName(), current_join_tree_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); @@ -3013,11 +3035,12 @@ void QueryAnalyzer::initializeTableExpressionColumns(QueryTreeNodePtr & table_ex { auto * table_node = table_expression_node->as(); auto * query_node = table_expression_node->as(); + auto * union_node = table_expression_node->as(); auto * table_function_node = table_expression_node->as(); - if (!table_node && !table_function_node && !query_node) + if (!table_node && !table_function_node && !query_node && !union_node) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Unexpected table expression. Expected table, table function or query node. Actual {}. In scope {}", + "Unexpected table expression. Expected table, table function, query or union node. Actual {}. In scope {}", table_expression_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); @@ -3084,9 +3107,9 @@ void QueryAnalyzer::initializeTableExpressionColumns(QueryTreeNodePtr & table_ex storage_columns.column_name_to_column_node = std::move(column_name_to_column_node); } - else if (query_node) + else if (query_node || union_node) { - auto column_names_and_types = query_node->computeProjectionColumns(); + auto column_names_and_types = query_node ? query_node->computeProjectionColumns() : union_node->computeProjectionColumns(); storage_columns.column_name_to_column_node.reserve(column_names_and_types.size()); for (const auto & column_name_and_type : column_names_and_types) @@ -3138,6 +3161,12 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, resolveQuery(join_tree_node, subquery_scope); break; } + case QueryTreeNodeType::UNION: + { + IdentifierResolveScope subquery_scope(join_tree_node, &scope); + resolveUnion(join_tree_node, subquery_scope); + break; + } case QueryTreeNodeType::TABLE_FUNCTION: { auto & table_function_node = join_tree_node->as(); @@ -3178,8 +3207,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, { if (argument_node->getNodeType() == QueryTreeNodeType::MATCHER) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Matcher as table function argument is not supported {}. In scope {}", join_tree_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); @@ -3341,7 +3369,11 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, } } - if (isTableExpression(join_tree_node.get())) + auto join_tree_node_type = join_tree_node->getNodeType(); + if (join_tree_node_type == QueryTreeNodeType::QUERY || + join_tree_node_type == QueryTreeNodeType::UNION || + join_tree_node_type == QueryTreeNodeType::TABLE || + join_tree_node_type == QueryTreeNodeType::TABLE_FUNCTION) initializeTableExpressionColumns(join_tree_node, scope); add_table_expression_alias_into_scope(join_tree_node); @@ -3419,17 +3451,17 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier */ scope.use_identifier_lookup_to_result_cache = false; - if (query_node_typed.getFrom()) + if (query_node_typed.getJoinTree()) { TableExpressionsAliasVisitor::Data table_expressions_visitor_data{scope}; TableExpressionsAliasVisitor table_expressions_visitor(table_expressions_visitor_data); - table_expressions_visitor.visit(query_node_typed.getFrom()); + table_expressions_visitor.visit(query_node_typed.getJoinTree()); - initializeQueryJoinTreeNode(query_node_typed.getFrom(), scope); + initializeQueryJoinTreeNode(query_node_typed.getJoinTree(), scope); scope.alias_name_to_table_expression_node.clear(); - resolveQueryJoinTreeNode(query_node_typed.getFrom(), scope, visitor); + resolveQueryJoinTreeNode(query_node_typed.getJoinTree(), scope, visitor); } scope.use_identifier_lookup_to_result_cache = true; @@ -3506,13 +3538,38 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier } } +void QueryAnalyzer::resolveUnion(const QueryTreeNodePtr & union_node, IdentifierResolveScope & scope) +{ + auto & union_node_typed = union_node->as(); + auto & queries_nodes = union_node_typed.getQueries().getNodes(); + + for (auto & query_node : queries_nodes) + { + IdentifierResolveScope subquery_scope(query_node, &scope /*parent_scope*/); + auto query_node_type = query_node->getNodeType(); + + if (query_node_type == QueryTreeNodeType::QUERY) + { + resolveQuery(query_node, subquery_scope); + } + else if (query_node_type == QueryTreeNodeType::UNION) + { + resolveUnion(query_node, subquery_scope); + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "UNION unsupported node {}. In scope {}", + query_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + } +} + } void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { - if (query_tree_node->getNodeType() != QueryTreeNodeType::QUERY) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "QueryAnalysis pass requires query node"); - QueryAnalyzer analyzer(std::move(context)); analyzer.resolve(query_tree_node); } diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 36972fed934..5f754e31d0b 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -63,10 +63,10 @@ String QueryNode::getName() const buffer << " SELECT "; buffer << getProjection().getName(); - if (getFrom()) + if (getJoinTree()) { buffer << " FROM "; - buffer << getFrom()->getName(); + buffer << getJoinTree()->getName(); } if (getPrewhere()) @@ -107,10 +107,10 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << std::string(indent + 2, ' ') << "PROJECTION\n"; getProjection().dumpTreeImpl(buffer, format_state, indent + 4); - if (getFrom()) + if (getJoinTree()) { buffer << '\n' << std::string(indent + 2, ' ') << "JOIN TREE\n"; - getFrom()->dumpTreeImpl(buffer, format_state, indent + 4); + getJoinTree()->dumpTreeImpl(buffer, format_state, indent + 4); } if (getPrewhere()) @@ -129,7 +129,7 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); - return is_subquery == rhs_typed.is_subquery && cte_name == rhs_typed.cte_name; + return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name; } void QueryNode::updateTreeHashImpl(HashState & state) const @@ -151,7 +151,7 @@ ASTPtr QueryNode::toASTImpl() const select_query->setExpression(ASTSelectQuery::Expression::SELECT, children[projection_child_index]->toAST()); ASTPtr tables_in_select_query_ast = std::make_shared(); - addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[from_child_index]); + addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, getJoinTree()); select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select_query_ast)); if (getPrewhere()) diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index a3b9ebaedd6..911e899ab42 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -87,14 +87,14 @@ public: return children[projection_child_index]; } - const QueryTreeNodePtr & getFrom() const + const QueryTreeNodePtr & getJoinTree() const { - return children[from_child_index]; + return children[join_tree_child_index]; } - QueryTreeNodePtr & getFrom() + QueryTreeNodePtr & getJoinTree() { - return children[from_child_index]; + return children[join_tree_child_index]; } const QueryTreeNodePtr & getPrewhere() const @@ -150,7 +150,7 @@ private: static constexpr size_t with_child_index = 0; static constexpr size_t projection_child_index = 1; - static constexpr size_t from_child_index = 2; + static constexpr size_t join_tree_child_index = 2; static constexpr size_t prewhere_child_index = 3; static constexpr size_t where_child_index = 4; static constexpr size_t group_by_child_index = 5; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index fa4f5b18c83..9164ac69a0a 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -33,6 +34,7 @@ #include #include #include +#include #include #include @@ -54,6 +56,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ + class QueryTreeBuilder : public WithContext { public: @@ -65,17 +70,21 @@ public: } private: - QueryTreeNodePtr getSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_subquery, const std::string & cte_name) const; + QueryTreeNodePtr buildSelectOrUnionExpression(const ASTPtr & select_or_union_query, bool is_subquery, const std::string & cte_name) const; - QueryTreeNodePtr getSelectExpression(const ASTPtr & select_query, bool is_subquery, const std::string & cte_name) const; + QueryTreeNodePtr buildSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_subquery, const std::string & cte_name) const; - QueryTreeNodePtr getExpressionList(const ASTPtr & expression_list) const; + QueryTreeNodePtr buildSelectIntersectExceptQuery(const ASTPtr & select_intersect_except_query, bool is_subquery, const std::string & cte_name) const; - QueryTreeNodePtr getExpression(const ASTPtr & expression) const; + QueryTreeNodePtr buildSelectExpression(const ASTPtr & select_query, bool is_subquery, const std::string & cte_name) const; - QueryTreeNodePtr getFromNode(const ASTPtr & tables_in_select_query) const; + QueryTreeNodePtr buildExpressionList(const ASTPtr & expression_list) const; - ColumnTransformersNodes getColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const; + QueryTreeNodePtr buildExpression(const ASTPtr & expression) const; + + QueryTreeNodePtr buildJoinTree(const ASTPtr & tables_in_select_query) const; + + ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const; ASTPtr query; QueryTreeNodePtr query_tree_node; @@ -86,47 +95,94 @@ QueryTreeBuilder::QueryTreeBuilder(ASTPtr query_, ContextPtr context_) : WithContext(context_) , query(query_->clone()) { - if (query->as()) - query_tree_node = getSelectWithUnionExpression(query, false /*is_subquery*/, {} /*cte_name*/); - else if (query->as()) - query_tree_node = getSelectExpression(query, false /*is_subquery*/, {} /*cte_name*/); + if (query->as() || + query->as() || + query->as()) + query_tree_node = buildSelectOrUnionExpression(query, false /*is_subquery*/, {} /*cte_name*/); else if (query->as()) - query_tree_node = getExpressionList(query); + query_tree_node = buildExpressionList(query); else - query_tree_node = getExpression(query); + query_tree_node = buildExpression(query); } -QueryTreeNodePtr QueryTreeBuilder::getSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_subquery, const std::string & cte_name) const +QueryTreeNodePtr QueryTreeBuilder::buildSelectOrUnionExpression(const ASTPtr & select_or_union_query, bool is_subquery, const std::string & cte_name) const +{ + QueryTreeNodePtr query_node; + + if (select_or_union_query->as()) + query_node = buildSelectWithUnionExpression(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/); + else if (select_or_union_query->as()) + query_node = buildSelectIntersectExceptQuery(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/); + else if (select_or_union_query->as()) + query_node = buildSelectExpression(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/); + else + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "UNION query {} is not supported", select_or_union_query->formatForErrorMessage()); + + return query_node; +} + +QueryTreeNodePtr QueryTreeBuilder::buildSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_subquery, const std::string & cte_name) const { auto & select_with_union_query_typed = select_with_union_query->as(); auto & select_lists = select_with_union_query_typed.list_of_selects->as(); if (select_lists.children.size() == 1) + return buildSelectOrUnionExpression(select_lists.children[0], is_subquery, cte_name); + + auto union_node = std::make_shared(); + union_node->setIsSubquery(is_subquery); + union_node->setCTEName(cte_name); + union_node->setUnionMode(select_with_union_query_typed.union_mode); + union_node->setUnionModes(select_with_union_query_typed.list_of_modes); + union_node->setOriginalAST(select_with_union_query); + + size_t select_lists_children_size = select_lists.children.size(); + + for (size_t i = 0; i < select_lists_children_size; ++i) { - return getSelectExpression(select_with_union_query->children[0]->children[0], is_subquery, cte_name); + auto & select_list_node = select_lists.children[i]; + QueryTreeNodePtr query_node = buildSelectOrUnionExpression(select_list_node, false /*is_subquery*/, {} /*cte_name*/); + union_node->getQueries().getNodes().push_back(std::move(query_node)); } - else - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "UNION is not supported"); - // auto union_expression = UnionExpression::create(is_scalar_query); - // union_expression->getModes() = select_with_union_query_typed.list_of_modes; - // union_expression->getModesSet() = select_with_union_query_typed.set_of_modes; - - // auto & select_expressions = union_expression->getSelectExpressions(); - // select_expressions.reserve(select_lists.children.size()); - - // for (const auto & select : select_lists.children) - // { - // auto expression = getSelectExpression(select, false); - // select_expressions.emplace_back(std::move(expression)); - // } - - // return union_expression; - } + return union_node; } -QueryTreeNodePtr QueryTreeBuilder::getSelectExpression(const ASTPtr & select_query, bool is_subquery, const std::string & cte_name) const +QueryTreeNodePtr QueryTreeBuilder::buildSelectIntersectExceptQuery(const ASTPtr & select_intersect_except_query, bool is_subquery, const std::string & cte_name) const +{ + auto & select_intersect_except_query_typed = select_intersect_except_query->as(); + auto select_lists = select_intersect_except_query_typed.getListOfSelects(); + + if (select_lists.size() == 1) + return buildSelectExpression(select_lists[0], is_subquery, cte_name); + + auto union_node = std::make_shared(); + union_node->setIsSubquery(is_subquery); + union_node->setCTEName(cte_name); + + if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::INTERSECT) + union_node->setUnionMode(SelectUnionMode::INTERSECT); + else if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT) + union_node->setUnionMode(SelectUnionMode::EXCEPT); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "UNION type is not initialized"); + + union_node->setUnionModes(SelectUnionModes(select_lists.size() - 1, union_node->getUnionMode())); + union_node->setOriginalAST(select_intersect_except_query); + + size_t select_lists_size = select_lists.size(); + + for (size_t i = 0; i < select_lists_size; ++i) + { + auto & select_list_node = select_lists[i]; + QueryTreeNodePtr query_node = buildSelectOrUnionExpression(select_list_node, false /*is_subquery*/, {} /*cte_name*/); + union_node->getQueries().getNodes().push_back(std::move(query_node)); + } + + return union_node; +} + +QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_query, bool is_subquery, const std::string & cte_name) const { const auto & select_query_typed = select_query->as(); auto current_query_tree = std::make_shared(); @@ -135,7 +191,7 @@ QueryTreeNodePtr QueryTreeBuilder::getSelectExpression(const ASTPtr & select_que current_query_tree->setIsCTE(!cte_name.empty()); current_query_tree->setCTEName(cte_name); - current_query_tree->getFrom() = getFromNode(select_query_typed.tables()); + current_query_tree->getJoinTree() = buildJoinTree(select_query_typed.tables()); current_query_tree->setOriginalAST(select_query); auto select_with_list = select_query_typed.with(); @@ -144,7 +200,7 @@ QueryTreeNodePtr QueryTreeBuilder::getSelectExpression(const ASTPtr & select_que auto & select_with_list_typed = select_with_list->as(); for (auto & expression_part : select_with_list_typed.children) { - auto expression_node = getExpression(expression_part); + auto expression_node = buildExpression(expression_part); current_query_tree->getWith().getNodes().push_back(expression_node); } } @@ -156,23 +212,23 @@ QueryTreeNodePtr QueryTreeBuilder::getSelectExpression(const ASTPtr & select_que for (auto & expression_part : select_expression_list_typed.children) { - auto expression_node = getExpression(expression_part); + auto expression_node = buildExpression(expression_part); current_query_tree->getProjection().getNodes().push_back(expression_node); } } auto prewhere_expression = select_query_typed.prewhere(); if (prewhere_expression) - current_query_tree->getPrewhere() = getExpression(prewhere_expression); + current_query_tree->getPrewhere() = buildExpression(prewhere_expression); auto where_expression = select_query_typed.where(); if (where_expression) - current_query_tree->getWhere() = getExpression(where_expression); + current_query_tree->getWhere() = buildExpression(where_expression); return current_query_tree; } -QueryTreeNodePtr QueryTreeBuilder::getExpressionList(const ASTPtr & expression_list) const +QueryTreeNodePtr QueryTreeBuilder::buildExpressionList(const ASTPtr & expression_list) const { auto list_node = std::make_shared(); @@ -181,14 +237,14 @@ QueryTreeNodePtr QueryTreeBuilder::getExpressionList(const ASTPtr & expression_l for (auto & expression : expression_list_typed.children) { - auto expression_node = getExpression(expression); + auto expression_node = buildExpression(expression); list_node->getNodes().push_back(std::move(expression_node)); } return list_node; } -QueryTreeNodePtr QueryTreeBuilder::getExpression(const ASTPtr & expression) const +QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) const { QueryTreeNodePtr result; @@ -200,7 +256,7 @@ QueryTreeNodePtr QueryTreeBuilder::getExpression(const ASTPtr & expression) cons } else if (const auto * asterisk = expression->as()) { - auto column_transformers = getColumnTransformers(expression, 0 /*start_child_index*/); + auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/); result = std::make_shared(column_transformers); } else if (const auto * qualified_asterisk = expression->as()) @@ -210,7 +266,7 @@ QueryTreeNodePtr QueryTreeBuilder::getExpression(const ASTPtr & expression) cons /// but ASTTableIdentifier can contain only 2 parts. auto & qualified_identifier = qualified_asterisk->children.at(0)->as(); - auto column_transformers = getColumnTransformers(expression, 1 /*start_child_index*/); + auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/); result = std::make_shared(Identifier(qualified_identifier.name_parts), column_transformers); } else if (const auto * ast_literal = expression->as()) @@ -259,7 +315,7 @@ QueryTreeNodePtr QueryTreeBuilder::getExpression(const ASTPtr & expression) cons } const auto & lambda_expression = lambda_arguments_and_expression.at(1); - auto lambda_expression_node = getExpression(lambda_expression); + auto lambda_expression_node = buildExpression(lambda_expression); result = std::make_shared(std::move(lambda_arguments), std::move(lambda_expression_node)); } @@ -271,14 +327,14 @@ QueryTreeNodePtr QueryTreeBuilder::getExpression(const ASTPtr & expression) cons { const auto & function_parameters_list = function->parameters->as()->children; for (const auto & argument : function_parameters_list) - function_node->getParameters().getNodes().push_back(getExpression(argument)); + function_node->getParameters().getNodes().push_back(buildExpression(argument)); } if (function->arguments) { const auto & function_arguments_list = function->arguments->as()->children; for (const auto & argument : function_arguments_list) - function_node->getArguments().getNodes().push_back(getExpression(argument)); + function_node->getArguments().getNodes().push_back(buildExpression(argument)); } result = function_node; @@ -287,20 +343,20 @@ QueryTreeNodePtr QueryTreeBuilder::getExpression(const ASTPtr & expression) cons else if (const auto * subquery = expression->as()) { auto subquery_query = subquery->children[0]; - auto query_node = getSelectWithUnionExpression(subquery_query, true /*is_subquery*/, {} /*cte_name*/); + auto query_node = buildSelectWithUnionExpression(subquery_query, true /*is_subquery*/, {} /*cte_name*/); result = query_node; } else if (const auto * with_element = expression->as()) { auto with_element_subquery = with_element->subquery->as().children.at(0); - auto query_node = getSelectWithUnionExpression(with_element_subquery, true /*is_subquery*/, with_element->name /*cte_name*/); + auto query_node = buildSelectWithUnionExpression(with_element_subquery, true /*is_subquery*/, with_element->name /*cte_name*/); result = query_node; } else if (const auto * columns_regexp_matcher = expression->as()) { - auto column_transformers = getColumnTransformers(expression, 0 /*start_child_index*/); + auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/); result = std::make_shared(columns_regexp_matcher->getMatcher(), std::move(column_transformers)); } else if (const auto * columns_list_matcher = expression->as()) @@ -314,13 +370,13 @@ QueryTreeNodePtr QueryTreeBuilder::getExpression(const ASTPtr & expression) cons column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts}); } - auto column_transformers = getColumnTransformers(expression, 0 /*start_child_index*/); + auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/); result = std::make_shared(std::move(column_list_identifiers), std::move(column_transformers)); } else if (const auto * qualified_columns_regexp_matcher = expression->as()) { auto & qualified_identifier = qualified_columns_regexp_matcher->children.at(0)->as(); - auto column_transformers = getColumnTransformers(expression, 1 /*start_child_index*/); + auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/); result = std::make_shared(Identifier(qualified_identifier.name_parts), qualified_columns_regexp_matcher->getMatcher(), std::move(column_transformers)); } else if (const auto * qualified_columns_list_matcher = expression->as()) @@ -336,7 +392,7 @@ QueryTreeNodePtr QueryTreeBuilder::getExpression(const ASTPtr & expression) cons column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts}); } - auto column_transformers = getColumnTransformers(expression, 1 /*start_child_index*/); + auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/); result = std::make_shared(Identifier(qualified_identifier.name_parts), column_list_identifiers, std::move(column_transformers)); } else @@ -350,7 +406,7 @@ QueryTreeNodePtr QueryTreeBuilder::getExpression(const ASTPtr & expression) cons return result; } -QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_query) const +QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select_query) const { if (!tables_in_select_query) { @@ -389,7 +445,7 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q auto & subquery_expression = table_expression.subquery->as(); const auto & select_with_union_query = subquery_expression.children[0]; - auto node = getSelectWithUnionExpression(select_with_union_query, true /*is_subquery*/, {} /*cte_name*/); + auto node = buildSelectWithUnionExpression(select_with_union_query, true /*is_subquery*/, {} /*cte_name*/); node->setAlias(subquery_expression.tryGetAlias()); node->setOriginalAST(select_with_union_query); @@ -406,7 +462,7 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q { const auto & function_arguments_list = table_function_expression.arguments->as()->children; for (const auto & argument : function_arguments_list) - node->getArguments().getNodes().push_back(getExpression(argument)); + node->getArguments().getNodes().push_back(buildExpression(argument)); } node->setAlias(table_function_expression.tryGetAlias()); @@ -433,9 +489,9 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q QueryTreeNodePtr join_expression; if (table_join.using_expression_list) - join_expression = getExpressionList(table_join.using_expression_list); + join_expression = buildExpressionList(table_join.using_expression_list); else if (table_join.on_expression) - join_expression = getExpression(table_join.on_expression); + join_expression = buildExpression(table_join.on_expression); auto join_node = std::make_shared(std::move(left_table_expression), std::move(right_table_expression), @@ -458,7 +514,7 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q auto last_table_expression = std::move(table_expressions.back()); table_expressions.pop_back(); - auto array_join_expressions_list = getExpressionList(array_join_expression.expression_list); + auto array_join_expressions_list = buildExpressionList(array_join_expression.expression_list); auto array_join_node = std::make_shared(std::move(last_table_expression), std::move(array_join_expressions_list), is_left_array_join); /** Original AST is not set because it will contain only array join part and does @@ -478,7 +534,7 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q } -ColumnTransformersNodes QueryTreeBuilder::getColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const +ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const { ColumnTransformersNodes column_transformers; size_t children_size = matcher_expression->children.size(); @@ -491,14 +547,14 @@ ColumnTransformersNodes QueryTreeBuilder::getColumnTransformers(const ASTPtr & m { if (apply_transformer->lambda) { - auto lambda_query_tree_node = getExpression(apply_transformer->lambda); + auto lambda_query_tree_node = buildExpression(apply_transformer->lambda); column_transformers.emplace_back(std::make_shared(std::move(lambda_query_tree_node))); } else { auto function_node = std::make_shared(apply_transformer->func_name); if (apply_transformer->parameters) - function_node->getParametersNode() = getExpressionList(apply_transformer->parameters); + function_node->getParametersNode() = buildExpressionList(apply_transformer->parameters); column_transformers.emplace_back(std::make_shared(std::move(function_node))); } @@ -529,7 +585,7 @@ ColumnTransformersNodes QueryTreeBuilder::getColumnTransformers(const ASTPtr & m for (const auto & replace_transformer_child : replace_transformer->children) { auto & replacement = replace_transformer_child->as(); - replacements.emplace_back(ReplaceColumnTransformerNode::Replacement{replacement.name, getExpression(replacement.expr)}); + replacements.emplace_back(ReplaceColumnTransformerNode::Replacement{replacement.name, buildExpression(replacement.expr)}); } column_transformers.emplace_back(std::make_shared(replacements, replace_transformer->is_strict)); @@ -543,6 +599,8 @@ ColumnTransformersNodes QueryTreeBuilder::getColumnTransformers(const ASTPtr & m return column_transformers; } +} + QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context) { QueryTreeBuilder builder(query, context); diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp new file mode 100644 index 00000000000..5af59f50aae --- /dev/null +++ b/src/Analyzer/UnionNode.cpp @@ -0,0 +1,193 @@ +#include + +#include + +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include + +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; +} + +UnionNode::UnionNode() +{ + children.resize(children_size); + children[queries_child_index] = std::make_shared(); +} + +NamesAndTypesList UnionNode::computeProjectionColumns() const +{ + std::vector projections; + const auto & query_nodes = getQueries().getNodes(); + + NamesAndTypes query_node_projection; + + for (const auto & query_node : query_nodes) + { + if (auto * query_node_typed = query_node->as()) + { + auto projection_columns = query_node_typed->computeProjectionColumns(); + query_node_projection = NamesAndTypes(projection_columns.begin(), projection_columns.end()); + } + else if (auto * union_node_typed = query_node->as()) + { + auto projection_columns = union_node_typed->computeProjectionColumns(); + query_node_projection = NamesAndTypes(projection_columns.begin(), projection_columns.end()); + } + + projections.push_back(query_node_projection); + + if (query_node_projection.size() != projections.front().size()) + throw Exception(ErrorCodes::TYPE_MISMATCH, "UNION different number of columns in queries"); + } + + NamesAndTypesList result_columns; + + size_t projections_size = projections.size(); + DataTypes projection_column_types; + projection_column_types.resize(projections_size); + + size_t columns_size = query_node_projection.size(); + for (size_t column_index = 0; column_index < columns_size; ++column_index) + { + for (size_t projection_index = 0; projection_index < projections_size; ++projection_index) + projection_column_types[projection_index] = projections[projection_index][column_index].type; + + auto result_type = getLeastSupertype(projection_column_types); + result_columns.emplace_back(projections.front()[column_index].name, std::move(result_type)); + } + + return result_columns; +} + +String UnionNode::getName() const +{ + WriteBufferFromOwnString buffer; + + auto query_nodes = getQueries().getNodes(); + size_t query_nodes_size = query_nodes.size(); + + for (size_t i = 0; i < query_nodes_size; ++i) + { + const auto & query_node = query_nodes[i]; + buffer << query_node->getName(); + + if (i == 0) + continue; + + auto query_union_mode = union_modes.at(i - 1); + if (query_union_mode == SelectUnionMode::ALL || query_union_mode == SelectUnionMode::DISTINCT) + buffer << " UNION " << toString(query_union_mode); + else + buffer << toString(query_union_mode); + } + + return buffer.str(); +} + +void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const +{ + buffer << std::string(indent, ' ') << "UNION id: " << format_state.getNodeId(this); + + if (hasAlias()) + buffer << ", alias: " << getAlias(); + + buffer << ", is_subquery: " << is_subquery; + buffer << ", is_cte: " << is_cte; + + if (!cte_name.empty()) + buffer << ", cte_name: " << cte_name; + + buffer << ", union_mode: "; + + if (union_mode == SelectUnionMode::ALL || union_mode == SelectUnionMode::DISTINCT) + buffer << " UNION " << toString(union_mode); + else + buffer << toString(union_mode); + + size_t union_modes_size = union_modes.size(); + buffer << '\n' << std::string(indent + 2, ' ') << "UNION MODES " << union_modes_size << '\n'; + + for (size_t i = 0; i < union_modes_size; ++i) + { + buffer << std::string(indent + 4, ' '); + + auto query_union_mode = union_modes[i]; + if (query_union_mode == SelectUnionMode::ALL || query_union_mode == SelectUnionMode::DISTINCT) + buffer << " UNION " << toString(query_union_mode); + else + buffer << toString(query_union_mode); + + if (i + 1 != union_modes_size) + buffer << '\n'; + } + + buffer << '\n' << std::string(indent + 2, ' ') << "QUERIES\n"; + getQueriesNode()->dumpTreeImpl(buffer, format_state, indent + 4); +} + +bool UnionNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name; +} + +void UnionNode::updateTreeHashImpl(HashState & state) const +{ + state.update(is_subquery); + state.update(is_cte); + + state.update(cte_name.size()); + state.update(cte_name); +} + +ASTPtr UnionNode::toASTImpl() const +{ + auto select_with_union_query = std::make_shared(); + select_with_union_query->union_mode = union_mode; + select_with_union_query->list_of_modes = union_modes; + select_with_union_query->set_of_modes = union_modes_set; + select_with_union_query->children.push_back(getQueriesNode()->toAST()); + select_with_union_query->list_of_selects = select_with_union_query->children.back(); + + return select_with_union_query; +} + +QueryTreeNodePtr UnionNode::cloneImpl() const +{ + auto result_query_node = std::make_shared(); + + result_query_node->is_subquery = is_subquery; + result_query_node->is_cte = is_cte; + result_query_node->cte_name = cte_name; + + result_query_node->union_mode = union_mode; + result_query_node->union_modes = union_modes; + result_query_node->union_modes_set = union_modes_set; + + return result_query_node; +} + +} diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h new file mode 100644 index 00000000000..94dbe30f470 --- /dev/null +++ b/src/Analyzer/UnionNode.h @@ -0,0 +1,126 @@ +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ + +/** Union node represents union of queries in query tree. + */ +class UnionNode; +using UnionNodePtr = std::shared_ptr; + +class UnionNode final : public IQueryTreeNode +{ +public: + explicit UnionNode(); + + bool isSubquery() const + { + return is_subquery; + } + + void setIsSubquery(bool is_subquery_value) + { + is_subquery = is_subquery_value; + } + + bool isCTE() const + { + return is_cte; + } + + void setIsCTE(bool is_cte_value) + { + is_cte = is_cte_value; + } + + const std::string & getCTEName() const + { + return cte_name; + } + + void setCTEName(std::string cte_name_value) + { + cte_name = std::move(cte_name_value); + } + + SelectUnionMode getUnionMode() const + { + return union_mode; + } + + void setUnionMode(SelectUnionMode union_mode_value) + { + union_mode = union_mode_value; + } + + const SelectUnionModes & getUnionModes() const + { + return union_modes; + } + + void setUnionModes(const SelectUnionModes & union_modes_value) + { + union_modes = union_modes_value; + union_modes_set = SelectUnionModesSet(union_modes.begin(), union_modes.end()); + } + + const QueryTreeNodePtr & getQueriesNode() const + { + return children[queries_child_index]; + } + + QueryTreeNodePtr & getQueriesNode() + { + return children[queries_child_index]; + } + + const ListNode & getQueries() const + { + return children[queries_child_index]->as(); + } + + ListNode & getQueries() + { + return children[queries_child_index]->as(); + } + + /// Compute union projection + NamesAndTypesList computeProjectionColumns() const; + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::UNION; + } + + String getName() const override; + + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + + void updateTreeHashImpl(HashState &) const override; + +protected: + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + bool is_subquery = false; + bool is_cte = false; + std::string cte_name; + SelectUnionMode union_mode; + SelectUnionModes union_modes; + SelectUnionModesSet union_modes_set; + + static constexpr size_t queries_child_index = 0; + static constexpr size_t children_size = queries_child_index + 1; +}; + +} diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 75b3f37aaad..73c305a5304 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -43,12 +43,6 @@ bool isNameOfInFunction(const std::string & function_name) return is_special_function_in; } -bool isTableExpression(const IQueryTreeNode * node) -{ - auto node_type = node->getNodeType(); - return node_type == QueryTreeNodeType::TABLE || node_type == QueryTreeNodeType::TABLE_FUNCTION || node_type == QueryTreeNodeType::QUERY; -} - static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expression_node) { ASTPtr table_expression_node_ast; diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 015233a05dc..19d5a20f994 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -11,9 +11,6 @@ bool isNodePartOfTree(const IQueryTreeNode * node, const IQueryTreeNode * root); /// Returns true if function name is name of IN function or its variations, false otherwise bool isNameOfInFunction(const std::string & function_name); -/// Returns true if node hase type table, table function, or query, false otherwise -bool isTableExpression(const IQueryTreeNode * node); - /** Add table expression in tables in select query children. * If table expression node is not of identifier node, table node, query node, table function node, join node or array join node type throws logical error exception. */ diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 53b745babb9..7e450ec3210 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -11,6 +11,8 @@ #include #include +#include + #include #include @@ -24,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -43,6 +46,9 @@ #include #include #include +#include +#include +#include #include #include @@ -81,6 +87,8 @@ namespace ErrorCodes * TODO: Support projections * TODO: Support read in order optimization * TODO: Simplify actions chain + * TODO: UNION storage limits + * TODO: Interpreter resources */ namespace @@ -101,6 +109,35 @@ namespace return query_pipeline_buffer.str(); } +Block getCommonHeaderForUnion(const Blocks & headers) +{ + size_t num_selects = headers.size(); + Block common_header = headers.front(); + size_t num_columns = common_header.columns(); + + for (size_t query_num = 1; query_num < num_selects; ++query_num) + { + if (headers[query_num].columns() != num_columns) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Different number of columns in UNION elements: {} and {}", + common_header.dumpNames(), + headers[query_num].dumpNames()); + } + + std::vector columns(num_selects); + + for (size_t column_num = 0; column_num < num_columns; ++column_num) + { + for (size_t i = 0; i < num_selects; ++i) + columns[i] = &headers[i].getByPosition(column_num); + + ColumnWithTypeAndName & result_elem = common_header.getByPosition(column_num); + result_elem = getLeastSuperColumn(columns); + } + + return common_header; +} + class ActionsChainNode; using ActionsChainNodePtr = std::unique_ptr; using ActionsChainNodes = std::vector; @@ -894,7 +931,7 @@ private: const auto & function_node = node->as(); WriteBufferFromOwnString buffer; - buffer << function_node.getFunctionName(); + buffer << "__function_" + function_node.getFunctionName(); const auto & function_parameters_nodes = function_node.getParameters().getNodes(); @@ -973,6 +1010,8 @@ public: { case QueryTreeNodeType::QUERY: [[fallthrough]]; + case QueryTreeNodeType::UNION: + [[fallthrough]]; case QueryTreeNodeType::TABLE: [[fallthrough]]; case QueryTreeNodeType::TABLE_FUNCTION: @@ -1055,9 +1094,10 @@ public: if (column_source_node_type != QueryTreeNodeType::TABLE && column_source_node_type != QueryTreeNodeType::TABLE_FUNCTION && - column_source_node_type != QueryTreeNodeType::QUERY) + column_source_node_type != QueryTreeNodeType::QUERY && + column_source_node_type != QueryTreeNodeType::UNION) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected table, table function or query column source. Actual {}", + "Expected table, table function, query or union column source. Actual {}", column_source_node->formatASTForErrorMessage()); auto [source_columns_set_it, inserted] = it->second.source_columns_names.insert(column_node->getColumnName()); @@ -1115,6 +1155,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, auto * table_node = table_expression->as(); auto * table_function_node = table_expression->as(); auto * query_node = table_expression->as(); + auto * union_node = table_expression->as(); QueryPlan query_plan; @@ -1168,7 +1209,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, query_plan.addStep(std::move(read_from_pipe)); } } - else if (query_node) + else if (query_node || union_node) { InterpreterSelectQueryAnalyzer interpeter(table_expression, select_query_options, planner_context.query_context); interpeter.initializeQueryPlanIfNeeded(); @@ -1176,7 +1217,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, } else { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected table, table function or query. Actual {}", table_expression->formatASTForErrorMessage()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected table, table function, query or union. Actual {}", table_expression->formatASTForErrorMessage()); } auto rename_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); @@ -1975,6 +2016,8 @@ QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, { case QueryTreeNodeType::QUERY: [[fallthrough]]; + case QueryTreeNodeType::UNION: + [[fallthrough]]; case QueryTreeNodeType::TABLE: [[fallthrough]]; case QueryTreeNodeType::TABLE_FUNCTION: @@ -2009,26 +2052,13 @@ InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( , query(query_) , select_query_options(select_query_options_) { - if (auto * select_with_union_query_typed = query->as()) + if (query->as() || query->as()) { - auto & select_lists = select_with_union_query_typed->list_of_selects->as(); - - if (select_lists.children.size() == 1) - { - query = select_lists.children[0]; - } - else - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "UNION is not supported"); - } } else if (auto * subquery = query->as()) { query = subquery->children[0]; } - else if (auto * select_query_typed = query_->as()) - { - } else { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, @@ -2052,10 +2082,11 @@ InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( , query_tree(query_tree_) , select_query_options(select_query_options_) { - if (query_tree_->getNodeType() != QueryTreeNodeType::QUERY) + if (query_tree->getNodeType() != QueryTreeNodeType::QUERY && + query_tree->getNodeType() != QueryTreeNodeType::UNION) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Expected query node. Actual {}", - query_tree_->formatASTForErrorMessage()); + "Expected QUERY or UNION node. Actual {}", + query_tree->formatASTForErrorMessage()); } @@ -2084,10 +2115,85 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() if (query_plan.isInitialized()) return; - auto & query_node = query_tree->as(); - auto current_context = getContext(); + if (auto * union_query_tree = query_tree->as()) + { + auto union_mode = union_query_tree->getUnionMode(); + if (union_mode == SelectUnionMode::Unspecified) + throw Exception(ErrorCodes::LOGICAL_ERROR, "UNION mode must be initialized"); + + std::vector> query_plans; + Blocks query_plans_headers; + + for (auto & query_node : union_query_tree->getQueries().getNodes()) + { + InterpreterSelectQueryAnalyzer interpeter(query_node, select_query_options, current_context); + interpeter.initializeQueryPlanIfNeeded(); + auto query_node_plan = std::make_unique(std::move(interpeter).extractQueryPlan()); + query_plans_headers.push_back(query_node_plan->getCurrentDataStream().header); + query_plans.push_back(std::move(query_node_plan)); + } + + Block union_common_header = getCommonHeaderForUnion(query_plans_headers); + DataStreams query_plans_streams; + query_plans_streams.reserve(query_plans.size()); + + for (auto & query_node_plan : query_plans) + { + if (blocksHaveEqualStructure(query_node_plan->getCurrentDataStream().header, union_common_header)) + continue; + + auto actions_dag = ActionsDAG::makeConvertingActions( + query_node_plan->getCurrentDataStream().header.getColumnsWithTypeAndName(), + union_common_header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + auto converting_step = std::make_unique(query_node_plan->getCurrentDataStream(), std::move(actions_dag)); + converting_step->setStepDescription("Conversion before UNION"); + query_node_plan->addStep(std::move(converting_step)); + + query_plans_streams.push_back(query_node_plan->getCurrentDataStream()); + } + + const auto & settings = current_context->getSettingsRef(); + auto max_threads = settings.max_threads; + + if (union_mode == SelectUnionMode::ALL || union_mode == SelectUnionMode::DISTINCT) + { + auto union_step = std::make_unique(std::move(query_plans_streams), max_threads); + query_plan.unitePlans(std::move(union_step), std::move(query_plans)); + + if (union_query_tree->getUnionMode() == SelectUnionMode::DISTINCT) + { + /// Add distinct transform + SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); + + auto distinct_step = std::make_unique( + query_plan.getCurrentDataStream(), + limits, + 0 /*limit hint*/, + query_plan.getCurrentDataStream().header.getNames(), + false /*pre distinct*/, + settings.optimize_distinct_in_order); + + query_plan.addStep(std::move(distinct_step)); + } + } + else if (union_mode == SelectUnionMode::INTERSECT || union_mode == SelectUnionMode::EXCEPT) + { + IntersectOrExceptStep::Operator intersect_or_except_operator = IntersectOrExceptStep::Operator::INTERSECT; + if (union_mode == SelectUnionMode::EXCEPT) + intersect_or_except_operator = IntersectOrExceptStep::Operator::EXCEPT; + + auto union_step = std::make_unique(std::move(query_plans_streams), intersect_or_except_operator, max_threads); + query_plan.unitePlans(std::move(union_step), std::move(query_plans)); + } + + return; + } + + auto & query_node = query_tree->as(); + SelectQueryInfo select_query_info; select_query_info.original_query = query; select_query_info.query = query; @@ -2096,13 +2202,13 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() planner_context.query_context = getContext(); CollectTableExpressionIdentifiersVisitor collect_table_expression_identifiers_visitor; - collect_table_expression_identifiers_visitor.visit(query_node.getFrom(), planner_context); + collect_table_expression_identifiers_visitor.visit(query_node.getJoinTree(), planner_context); CollectSourceColumnsVisitor::Data data {planner_context}; CollectSourceColumnsVisitor collect_source_columns_visitor(data); collect_source_columns_visitor.visit(query_tree); - query_plan = buildQueryPlanForJoinTreeNode(query_node.getFrom(), select_query_info, select_query_options, planner_context); + query_plan = buildQueryPlanForJoinTreeNode(query_node.getJoinTree(), select_query_info, select_query_options, planner_context); std::optional> action_chain_node_parent_indices; if (query_node.hasWhere()) @@ -2167,6 +2273,14 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() planner_context.projection_actions->project(projection_names); + // std::cout << "Chain dump before finalize" << std::endl; + // std::cout << planner_context.actions_chain.dump() << std::endl; + + planner_context.actions_chain.finalize(); + + // std::cout << "Chain dump after finalize" << std::endl; + // std::cout << planner_context.actions_chain.dump() << std::endl; + if (query_node.hasWhere()) { auto & where_actions_chain_node = planner_context.actions_chain.at(planner_context.where_actions_chain_node_index); @@ -2185,14 +2299,6 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), planner_context.projection_actions); projection_step->setStepDescription("Projection"); query_plan.addStep(std::move(projection_step)); - - // std::cout << "Chain dump before finalize" << std::endl; - // std::cout << planner_context.actions_chain.dump() << std::endl; - - planner_context.actions_chain.finalize(); - - // std::cout << "Chain dump after finalize" << std::endl; - // std::cout << planner_context.actions_chain.dump() << std::endl; } } diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index 62e23202031..229cc94f6ff 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -87,7 +87,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected explain_query->setTableFunction(table_function); explain_query->setTableOverride(table_override); } - if (kind == ASTExplainQuery::ExplainKind::QueryTree) + else if (kind == ASTExplainQuery::ExplainKind::QueryTree) { if (select_p.parse(pos, query, expected)) explain_query->setExplainedQuery(std::move(query)); diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h index b2738cb297f..d7eab574431 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.h +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -8,9 +8,9 @@ namespace DB class IntersectOrExceptStep : public IQueryPlanStep { -using Operator = ASTSelectIntersectExceptQuery::Operator; - public: + using Operator = ASTSelectIntersectExceptQuery::Operator; + /// max_threads is used to limit the number of threads for result pipeline. IntersectOrExceptStep(DataStreams input_streams_, Operator operator_, size_t max_threads_ = 0); diff --git a/tests/queries/0_stateless/02375_analyzer_union.reference b/tests/queries/0_stateless/02375_analyzer_union.reference new file mode 100644 index 00000000000..199b9af5313 --- /dev/null +++ b/tests/queries/0_stateless/02375_analyzer_union.reference @@ -0,0 +1,62 @@ +-- { echoOn } + +SELECT 'Union constants'; +Union constants +SELECT 1 UNION ALL SELECT 1; +1 +1 +SELECT '--'; +-- +SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1; +1 +1 +SELECT '--'; +-- +SELECT 1 INTERSECT SELECT 1; +1 +SELECT '--'; +-- +SELECT 1 EXCEPT SELECT 1; +SELECT '--'; +-- +SELECT id FROM (SELECT 1 AS id UNION ALL SELECT 1); +1 +1 +SELECT 'Union non constants'; +Union non constants +SELECT value FROM (SELECT 1 as value UNION ALL SELECT 1 UNION ALL SELECT 1); +1 +1 +1 +SELECT '--'; +-- +SELECT id FROM test_table UNION ALL SELECT id FROM test_table; +0 +0 +SELECT '--'; +-- +SELECT id FROM test_table UNION DISTINCT SELECT id FROM test_table; +0 +SELECT '--'; +-- +SELECT id FROM test_table INTERSECT SELECT id FROM test_table; +0 +SELECT '--'; +-- +SELECT id FROM test_table EXCEPT SELECT id FROM test_table; +SELECT '--'; +-- +SELECT id FROM (SELECT id FROM test_table UNION ALL SELECT id FROM test_table); +0 +0 +SELECT '--'; +-- +SELECT id FROM (SELECT id FROM test_table UNION DISTINCT SELECT id FROM test_table); +0 +SELECT '--'; +-- +SELECT id FROM (SELECT id FROM test_table INTERSECT SELECT id FROM test_table); +0 +SELECT '--'; +-- +SELECT id FROM (SELECT id FROM test_table EXCEPT SELECT id FROM test_table); diff --git a/tests/queries/0_stateless/02375_analyzer_union.sql b/tests/queries/0_stateless/02375_analyzer_union.sql new file mode 100644 index 00000000000..4983040bd23 --- /dev/null +++ b/tests/queries/0_stateless/02375_analyzer_union.sql @@ -0,0 +1,71 @@ +SET use_analyzer = 0; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +-- { echoOn } + +SELECT 'Union constants'; + +SELECT 1 UNION ALL SELECT 1; + +SELECT '--'; + +SELECT 1 UNION DISTINCT SELECT 1 UNION ALL SELECT 1; + +SELECT '--'; + +SELECT 1 INTERSECT SELECT 1; + +SELECT '--'; + +SELECT 1 EXCEPT SELECT 1; + +SELECT '--'; + +SELECT id FROM (SELECT 1 AS id UNION ALL SELECT 1); + +SELECT 'Union non constants'; + +SELECT value FROM (SELECT 1 as value UNION ALL SELECT 1 UNION ALL SELECT 1); + +SELECT '--'; + +SELECT id FROM test_table UNION ALL SELECT id FROM test_table; + +SELECT '--'; + +SELECT id FROM test_table UNION DISTINCT SELECT id FROM test_table; + +SELECT '--'; + +SELECT id FROM test_table INTERSECT SELECT id FROM test_table; + +SELECT '--'; +SELECT id FROM test_table EXCEPT SELECT id FROM test_table; + +SELECT '--'; + +SELECT id FROM (SELECT id FROM test_table UNION ALL SELECT id FROM test_table); + +SELECT '--'; + +SELECT id FROM (SELECT id FROM test_table UNION DISTINCT SELECT id FROM test_table); + +SELECT '--'; + +SELECT id FROM (SELECT id FROM test_table INTERSECT SELECT id FROM test_table); + +SELECT '--'; + +SELECT id FROM (SELECT id FROM test_table EXCEPT SELECT id FROM test_table); + +-- { echoOff } + +DROP TABLE test_table; From 44ada63a97eed01638a3214d7db34b58d7119918 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 24 Aug 2022 11:21:03 +0200 Subject: [PATCH 072/188] Added Planner --- src/Analyzer/CMakeLists.txt | 2 +- src/CMakeLists.txt | 2 + src/Interpreters/InterpreterExplainQuery.cpp | 2 - .../InterpreterSelectQueryAnalyzer.cpp | 2211 +---------------- .../InterpreterSelectQueryAnalyzer.h | 22 +- src/Planner/ActionsChain.cpp | 124 + src/Planner/ActionsChain.h | 231 ++ src/Planner/CMakeLists.txt | 7 + src/Planner/Planner.cpp | 889 +++++++ src/Planner/Planner.h | 57 + src/Planner/PlannerActionsVisitor.cpp | 466 ++++ src/Planner/PlannerActionsVisitor.h | 50 + src/Planner/PlannerContext.cpp | 93 + src/Planner/PlannerContext.h | 218 ++ src/Planner/PlannerJoins.cpp | 441 ++++ src/Planner/PlannerJoins.h | 149 ++ src/Planner/Utils.cpp | 59 + src/Planner/Utils.h | 18 + src/Planner/examples/CMakeLists.txt | 0 src/Planner/tests/CMakeLists.txt | 0 20 files changed, 2837 insertions(+), 2204 deletions(-) create mode 100644 src/Planner/ActionsChain.cpp create mode 100644 src/Planner/ActionsChain.h create mode 100644 src/Planner/CMakeLists.txt create mode 100644 src/Planner/Planner.cpp create mode 100644 src/Planner/Planner.h create mode 100644 src/Planner/PlannerActionsVisitor.cpp create mode 100644 src/Planner/PlannerActionsVisitor.h create mode 100644 src/Planner/PlannerContext.cpp create mode 100644 src/Planner/PlannerContext.h create mode 100644 src/Planner/PlannerJoins.cpp create mode 100644 src/Planner/PlannerJoins.h create mode 100644 src/Planner/Utils.cpp create mode 100644 src/Planner/Utils.h create mode 100644 src/Planner/examples/CMakeLists.txt create mode 100644 src/Planner/tests/CMakeLists.txt diff --git a/src/Analyzer/CMakeLists.txt b/src/Analyzer/CMakeLists.txt index 1068fee4cea..766767b5c13 100644 --- a/src/Analyzer/CMakeLists.txt +++ b/src/Analyzer/CMakeLists.txt @@ -4,4 +4,4 @@ endif() if (ENABLE_EXAMPLES) add_subdirectory(examples) -endif() \ No newline at end of file +endif() diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index aa681126c73..2c54d1108d4 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -67,6 +67,7 @@ add_subdirectory (Parsers) add_subdirectory (IO) add_subdirectory (Functions) add_subdirectory (Analyzer) +add_subdirectory (Planner) add_subdirectory (Interpreters) add_subdirectory (AggregateFunctions) add_subdirectory (Client) @@ -256,6 +257,7 @@ add_object_library(clickhouse_databases Databases) add_object_library(clickhouse_databases_mysql Databases/MySQL) add_object_library(clickhouse_disks Disks) add_object_library(clickhouse_analyzer Analyzer) +add_object_library(clickhouse_planner Planner) add_object_library(clickhouse_interpreters Interpreters) add_object_library(clickhouse_interpreters_cache Interpreters/Cache) add_object_library(clickhouse_interpreters_access Interpreters/Access) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index d3d13d80f57..7428d69bdee 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -421,7 +421,6 @@ QueryPipeline InterpreterExplainQuery::executeImpl() if (getContext()->getSettingsRef().use_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), options, getContext()); - interpreter.initializeQueryPlanIfNeeded(); plan = std::move(interpreter).extractQueryPlan(); } else @@ -465,7 +464,6 @@ QueryPipeline InterpreterExplainQuery::executeImpl() if (getContext()->getSettingsRef().use_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), options, getContext()); - interpreter.initializeQueryPlanIfNeeded(); plan = std::move(interpreter).extractQueryPlan(); } else diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 7e450ec3210..b236af15df8 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -69,1977 +69,36 @@ namespace ErrorCodes extern const int INVALID_JOIN_ON_EXPRESSION; } -/** ClickHouse query planner. - * - * TODO: JOIN support ASOF. JOIN support strictness. JOIN support constants. JOIN support ON t1.id = t1.id - * TODO: JOIN drop unnecessary columns after ON, USING section - * TODO: Support display names - * TODO: Support RBAC. Support RBAC for ALIAS columns. - * TODO: Support distributed query processing - * TODO: Support PREWHERE - * TODO: Support GROUP BY, HAVING - * TODO: Support ORDER BY, LIMIT - * TODO: Support WINDOW FUNCTIONS - * TODO: Support DISTINCT - * TODO: Support building sets for IN functions - * TODO: Support trivial count optimization - * TODO: Support totals, extremes - * TODO: Support projections - * TODO: Support read in order optimization - * TODO: Simplify actions chain - * TODO: UNION storage limits - * TODO: Interpreter resources - */ - namespace { -[[maybe_unused]] String dumpQueryPlan(QueryPlan & query_plan) +ASTPtr normalizeAndValidateQuery(const ASTPtr & query) { - WriteBufferFromOwnString query_plan_buffer; - query_plan.explainPlan(query_plan_buffer, QueryPlan::ExplainPlanOptions{true, true, true, true}); - return query_plan_buffer.str(); -} - -[[maybe_unused]] String dumpQueryPipeline(QueryPlan & query_plan) -{ - QueryPlan::ExplainPipelineOptions explain_pipeline; - WriteBufferFromOwnString query_pipeline_buffer; - query_plan.explainPipeline(query_pipeline_buffer, explain_pipeline); - return query_pipeline_buffer.str(); -} - -Block getCommonHeaderForUnion(const Blocks & headers) -{ - size_t num_selects = headers.size(); - Block common_header = headers.front(); - size_t num_columns = common_header.columns(); - - for (size_t query_num = 1; query_num < num_selects; ++query_num) + if (query->as() || query->as()) { - if (headers[query_num].columns() != num_columns) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Different number of columns in UNION elements: {} and {}", - common_header.dumpNames(), - headers[query_num].dumpNames()); + return query; } - - std::vector columns(num_selects); - - for (size_t column_num = 0; column_num < num_columns; ++column_num) + else if (auto * subquery = query->as()) { - for (size_t i = 0; i < num_selects; ++i) - columns[i] = &headers[i].getByPosition(column_num); - - ColumnWithTypeAndName & result_elem = common_header.getByPosition(column_num); - result_elem = getLeastSuperColumn(columns); - } - - return common_header; -} - -class ActionsChainNode; -using ActionsChainNodePtr = std::unique_ptr; -using ActionsChainNodes = std::vector; - -class ActionsChainNode -{ -public: - explicit ActionsChainNode(ActionsDAGPtr actions_, bool available_output_columns_only_aliases_ = false) - : actions(std::move(actions_)) - , available_output_columns_only_aliases(available_output_columns_only_aliases_) - { - initialize(); - } - - [[maybe_unused]] ActionsDAGPtr & getActions() - { - return actions; - } - - [[maybe_unused]] const ActionsDAGPtr & getActions() const - { - return actions; - } - - const ColumnsWithTypeAndName & getAvailableOutputColumns() const - { - return available_output_columns; - } - - const NameSet & getInputColumnNames() const - { - return input_columns_names; - } - - const NameSet & getChildRequiredOutputColumnsNames() const - { - return child_required_output_columns_names; - } - - void finalizeInputAndOutputColumns(NameSet & child_input_columns) - { - child_required_output_columns_names.clear(); - std::vector required_output_nodes; - - for (const auto & node : actions->getNodes()) - { - auto it = child_input_columns.find(node.result_name); - - if (it == child_input_columns.end()) - continue; - - child_required_output_columns_names.insert(node.result_name); - required_output_nodes.push_back(&node); - child_input_columns.erase(it); - } - - for (auto & required_output_node : required_output_nodes) - actions->addOrReplaceInOutputs(*required_output_node); - - actions->removeUnusedActions(); - initialize(); - } - - void dump(WriteBuffer & buffer) const - { - buffer << "DAG" << '\n'; - buffer << actions->dumpDAG(); - if (!child_required_output_columns_names.empty()) - { - buffer << "Child required output columns " << boost::join(child_required_output_columns_names, ", "); - buffer << '\n'; - } - } - - [[maybe_unused]] String dump() const - { - WriteBufferFromOwnString buffer; - dump(buffer); - - return buffer.str(); - } - - [[maybe_unused]] void addParentIndex(size_t parent_node_index) - { - parent_nodes_indices.push_back(parent_node_index); - } - - void addParentIndices(const std::vector & parent_nodes_indices_value) - { - parent_nodes_indices.insert(parent_nodes_indices.end(), parent_nodes_indices_value.begin(), parent_nodes_indices_value.end()); - } - - const std::vector & getParentNodesIndices() const - { - return parent_nodes_indices; - } - -private: - void initialize() - { - auto required_columns_names = actions->getRequiredColumnsNames(); - input_columns_names = NameSet(required_columns_names.begin(), required_columns_names.end()); - - available_output_columns.clear(); - - for (const auto & node : actions->getNodes()) - { - if (available_output_columns_only_aliases) - { - if (node.type == ActionsDAG::ActionType::ALIAS) - available_output_columns.emplace_back(node.column, node.result_type, node.result_name); - - continue; - } - - if (node.type == ActionsDAG::ActionType::INPUT || - node.type == ActionsDAG::ActionType::FUNCTION || - node.type == ActionsDAG::ActionType::ARRAY_JOIN) - available_output_columns.emplace_back(node.column, node.result_type, node.result_name); - } - } - - ActionsDAGPtr actions; - - bool available_output_columns_only_aliases; - - NameSet input_columns_names; - - NameSet child_required_output_columns_names; - - ColumnsWithTypeAndName available_output_columns; - - std::vector parent_nodes_indices; - -}; - -class ActionsChain -{ -public: - void addNode(ActionsChainNodePtr node) - { - nodes.emplace_back(std::move(node)); - } - - [[maybe_unused]] const ActionsChainNodes & getNodes() const - { - return nodes; - } - - ColumnsWithTypeAndName getAvailableOutputColumns(const std::vector & nodes_indices) - { - ColumnsWithTypeAndName result; - - for (const auto & node_index : nodes_indices) - { - assert(node_index < nodes.size()); - const auto & node_available_output_columns = nodes[node_index]->getAvailableOutputColumns(); - result.insert(result.end(), node_available_output_columns.begin(), node_available_output_columns.end()); - } - - return result; - } - - // ColumnsWithTypeAndName getOutputColumns(const std::vector & nodes_indices) - // { - // ColumnsWithTypeAndName result; - - // for (const auto & node_index : nodes_indices) - // { - // assert(node_index < nodes.size()); - // const auto & node_output_columns = nodes[node_index]->getActions()->getResultColumns(); - // result.insert(result.end(), node_output_columns.begin(), node_output_columns.end()); - // } - - // return result; - // } - - [[maybe_unused]] NameSet getInputColumnNames(const std::vector & nodes_indices) - { - NameSet result; - - for (const auto & node_index : nodes_indices) - { - assert(node_index < nodes.size()); - const auto & node_input_column_names = nodes[node_index]->getInputColumnNames(); - result.insert(node_input_column_names.begin(), node_input_column_names.end()); - } - - return result; - } - - [[maybe_unused]] size_t size() const - { - return nodes.size(); - } - - [[maybe_unused]] const ActionsChainNodePtr & at(size_t index) const - { - if (index >= nodes.size()) - throw std::out_of_range("actions chain access is out of range"); - - return nodes[index]; - } - - [[maybe_unused]] ActionsChainNodePtr & at(size_t index) - { - if (index >= nodes.size()) - throw std::out_of_range("actions chain access is out of range"); - - return nodes[index]; - } - - [[maybe_unused]] ActionsChainNodePtr & operator[](size_t index) - { - return nodes[index]; - } - - [[maybe_unused]] const ActionsChainNodePtr & operator[](size_t index) const - { - return nodes[index]; - } - - [[maybe_unused]] ActionsChainNode * getLastNode() - { - return nodes.back().get(); - } - - [[maybe_unused]] ActionsChainNode * getLastNodeOrThrow() - { - if (nodes.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "ActionsChain is empty"); - - return nodes.back().get(); - } - - size_t getLastNodeIndex() - { - return nodes.size() - 1; - } - - [[maybe_unused]] size_t getLastNodeIndexOrThrow() - { - if (nodes.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "ActionsChain is empty"); - - return nodes.size() - 1; - } - - void finalize() - { - if (nodes.empty()) - return; - - std::deque nodes_indices_to_process; - nodes_indices_to_process.push_front(nodes.size() - 1); - - /// For root node there are no columns required in child nodes - NameSet empty_child_input_columns; - nodes.back().get()->finalizeInputAndOutputColumns(empty_child_input_columns); - - while (!nodes_indices_to_process.empty()) - { - auto node_index_to_process = nodes_indices_to_process.front(); - nodes_indices_to_process.pop_front(); - - auto & node_to_process = nodes[node_index_to_process]; - - const auto & parent_nodes_indices = node_to_process->getParentNodesIndices(); - auto input_columns_names_copy = node_to_process->getInputColumnNames(); - - for (const auto & parent_node_index : parent_nodes_indices) - { - assert(parent_node_index < nodes.size()); - - auto & parent_node = nodes[parent_node_index]; - parent_node->finalizeInputAndOutputColumns(input_columns_names_copy); - nodes_indices_to_process.push_back(parent_node_index); - } - } - } - - void dump(WriteBuffer & buffer) const - { - size_t nodes_size = nodes.size(); - - for (size_t i = 0; i < nodes_size; ++i) - { - const auto & node = nodes[i]; - buffer << "Node " << i; - - const auto & parent_nodes_indices = node->getParentNodesIndices(); - if (!parent_nodes_indices.empty()) - { - buffer << " parent nodes indices "; - for (const auto & parent_node_index : parent_nodes_indices) - buffer << parent_node_index << ' '; - } - - buffer << '\n'; - node->dump(buffer); - - buffer << '\n'; - } - } - - [[maybe_unused]] String dump() const - { - WriteBufferFromOwnString buffer; - dump(buffer); - return buffer.str(); - } - -private: - ActionsChainNodes nodes; -}; - -using ColumnIdentifier = std::string; - -struct TableExpressionColumns -{ - /// Valid for table, table function, query table expression nodes - NamesAndTypesList source_columns; - - /// Valid for table, table function, query table expression nodes - NameSet source_columns_names; - - /// Valid only for table table expression node - NameSet alias_columns; - - /// Valid for table, table function, query table expression nodes - std::unordered_map column_name_to_column_identifier; -}; - -using TableExpressionNodeToColumns = std::unordered_map; - -struct PlannerContext -{ - std::unordered_map column_node_to_column_identifier; - std::unordered_map table_expression_node_to_identifier; - - TableExpressionNodeToColumns table_expression_node_to_columns; - size_t column_identifier_counter = 0; - - ActionsChain actions_chain; - - ActionsDAGPtr where_actions; - size_t where_actions_chain_node_index = 0; - std::string where_action_node_name; - ActionsDAGPtr projection_actions; - - ContextPtr query_context; - - ColumnIdentifier getColumnUniqueIdentifier(const IQueryTreeNode * column_source_node, std::string column_name = {}) - { - auto column_unique_prefix = "__column_" + std::to_string(column_identifier_counter); - ++column_identifier_counter; - - std::string table_expression_identifier; - auto table_expression_identifier_it = table_expression_node_to_identifier.find(column_source_node); - if (table_expression_identifier_it != table_expression_node_to_identifier.end()) - table_expression_identifier = table_expression_identifier_it->second; - - std::string debug_identifier_suffix; - - if (column_source_node->hasAlias()) - { - debug_identifier_suffix += column_source_node->getAlias(); - } - else if (const auto * table_source_node = column_source_node->as()) - { - debug_identifier_suffix += table_source_node->getStorageID().getFullNameNotQuoted(); - } - else - { - auto column_source_node_type = column_source_node->getNodeType(); - if (column_source_node_type == QueryTreeNodeType::JOIN) - debug_identifier_suffix += "join"; - else if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN) - debug_identifier_suffix += "array_join"; - else if (column_source_node_type == QueryTreeNodeType::TABLE_FUNCTION) - debug_identifier_suffix += "table_function"; - else if (column_source_node_type == QueryTreeNodeType::QUERY) - debug_identifier_suffix += "subquery"; - - if (!table_expression_identifier.empty()) - debug_identifier_suffix += '_' + table_expression_identifier; - } - - if (!column_name.empty()) - debug_identifier_suffix += '.' + column_name; - - if (!debug_identifier_suffix.empty()) - column_unique_prefix += '_' + debug_identifier_suffix; - - return column_unique_prefix; - } - - ColumnIdentifier getColumnIdentifierOrThrow(const IQueryTreeNode * column_source_node) - { - assert(column_source_node->getNodeType() == QueryTreeNodeType::COLUMN); - auto it = column_node_to_column_identifier.find(column_source_node); - if (it == column_node_to_column_identifier.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Column identifier is not initialized for column {}", - column_source_node->formatASTForErrorMessage()); - - return it->second; - } -}; - -struct QueryTreeActionsScopeNode -{ - explicit QueryTreeActionsScopeNode(ActionsDAGPtr actions_dag_, QueryTreeNodePtr scope_node_) - : actions_dag(std::move(actions_dag_)) - , scope_node(std::move(scope_node_)) - { - for (const auto & node : actions_dag->getNodes()) - node_name_to_node[node.result_name] = &node; - } - - [[maybe_unused]] bool containsNode(const std::string & node_name) - { - return node_name_to_node.find(node_name) != node_name_to_node.end(); - } - - [[maybe_unused]] const ActionsDAG::Node * tryGetNode(const std::string & node_name) - { - auto it = node_name_to_node.find(node_name); - if (it == node_name_to_node.end()) - return {}; - - return it->second; - } - - const ActionsDAG::Node * getNodeOrThrow(const std::string & node_name) - { - auto it = node_name_to_node.find(node_name); - if (it == node_name_to_node.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "No node with name {}. There are only nodes {}", - node_name, - actions_dag->dumpNames()); - - return it->second; - } - - const ActionsDAG::Node * addInputColumnIfNecessary(const std::string & node_name, const DataTypePtr & column_type) - { - auto it = node_name_to_node.find(node_name); - if (it != node_name_to_node.end()) - return it->second; - - const auto * node = &actions_dag->addInput(node_name, column_type); - - // std::cout << "QueryTreeActionsScopeNode::addInputColumnIfNecessary dag " << actions_dag << " node name " << node_name; - // std::cout << " result node ptr " << node << std::endl; - - node_name_to_node[node->result_name] = node; - - return node; - } - - const ActionsDAG::Node * addInputConstantColumnIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) - { - auto it = node_name_to_node.find(node_name); - if (it != node_name_to_node.end()) - return it->second; - - const auto * node = &actions_dag->addInput(column); - node_name_to_node[node->result_name] = node; - - return node; - } - - const ActionsDAG::Node * addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) - { - auto it = node_name_to_node.find(node_name); - if (it != node_name_to_node.end()) - return it->second; - - const auto * node = &actions_dag->addColumn(column); - - // std::cout << "QueryTreeActionsScopeNode::addConstantIfNecessary dag " << actions_dag << " node name " << node_name; - // std::cout << " result node ptr " << node << std::endl; - - node_name_to_node[node->result_name] = node; - - return node; - } - - const ActionsDAG::Node * addFunctionIfNecessary(const std::string & node_name, ActionsDAG::NodeRawConstPtrs children, FunctionOverloadResolverPtr function) - { - auto it = node_name_to_node.find(node_name); - if (it != node_name_to_node.end()) - return it->second; - - const auto * node = &actions_dag->addFunction(function, children, node_name); - - // std::cout << "QueryTreeActionsScopeNode::addFunctionIfNecessary dag " << actions_dag << " node name " << node_name; - // std::cout << " result node ptr " << node << std::endl; - - node_name_to_node[node->result_name] = node; - - return node; - } - - const ActionsDAG::Node * addArrayJoinIfNecessary(const std::string & node_name, const ActionsDAG::Node * child) - { - auto it = node_name_to_node.find(node_name); - if (it != node_name_to_node.end()) - return it->second; - - const auto * node = &actions_dag->addArrayJoin(*child, node_name); - node_name_to_node[node->result_name] = node; - - return node; - } - - std::unordered_map node_name_to_node; - ActionsDAGPtr actions_dag; - QueryTreeNodePtr scope_node; -}; - -class QueryTreeActionsVisitor -{ -public: - explicit QueryTreeActionsVisitor(ActionsDAGPtr actions_dag, const PlannerContext & planner_context_) - : planner_context(planner_context_) - { - actions_stack.emplace_back(std::move(actions_dag), nullptr); - } - - ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node) - { - ActionsDAG::NodeRawConstPtrs result; - - if (auto * expression_list_node = expression_node->as()) - { - for (auto & node : expression_list_node->getNodes()) - { - auto [node_name, _] = visitImpl(node); - result.push_back(actions_stack.front().getNodeOrThrow(node_name)); - } - } - else - { - auto [node_name, _] = visitImpl(expression_node); - result.push_back(actions_stack.front().getNodeOrThrow(node_name)); - } - - return result; - } - -private: - - using NodeNameAndNodeMinLevel = std::pair; - - NodeNameAndNodeMinLevel visitImpl(QueryTreeNodePtr node) - { - if (auto * column_node = node->as()) - return visitColumn(node); - else if (auto * constant_node = node->as()) - return visitConstant(node); - else if (auto * function_node = node->as()) - return visitFunction(node); - - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected only column, constant or function node. Actual {}", node->formatASTForErrorMessage()); - } - - NodeNameAndNodeMinLevel visitColumn(const QueryTreeNodePtr & node) - { - auto column_node_name = getActionsDAGNodeName(node.get()); - const auto & column_node = node->as(); - - Int64 actions_stack_size = static_cast(actions_stack.size() - 1); - for (Int64 i = actions_stack_size; i >= 0; --i) - { - actions_stack[i].addInputColumnIfNecessary(column_node_name, column_node.getColumnType()); - - if (column_node.getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA && - actions_stack[i].scope_node.get() == column_node.getColumnSource().get()) - { - return {column_node_name, i}; - } - } - - return {column_node_name, 0}; - } - - NodeNameAndNodeMinLevel visitConstant(const QueryTreeNodePtr & node) - { - auto constant_node_name = getActionsDAGNodeName(node.get()); - const auto & constant_node = node->as(); - const auto & literal = constant_node.getConstantValue(); - - ColumnWithTypeAndName column; - column.name = constant_node_name; - column.type = constant_node.getResultType(); - column.column = column.type->createColumnConst(1, literal); - - actions_stack[0].addConstantIfNecessary(constant_node_name, column); - - size_t actions_stack_size = actions_stack.size(); - for (size_t i = 1; i < actions_stack_size; ++i) - { - auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputConstantColumnIfNecessary(constant_node_name, column); - } - - return {constant_node_name, 0}; - } - - NodeNameAndNodeMinLevel visitLambda(const QueryTreeNodePtr & node) - { - auto & lambda_node = node->as(); - auto result_type = lambda_node.getResultType(); - if (!result_type) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Lambda {} is not resolved during query analysis", - lambda_node.formatASTForErrorMessage()); - - NamesAndTypesList lambda_arguments_names_and_types; - - for (auto & lambda_node_argument : lambda_node.getArguments().getNodes()) - { - auto lambda_argument_name = lambda_node_argument->getName(); - auto lambda_argument_type = lambda_node_argument->getResultType(); - lambda_arguments_names_and_types.emplace_back(lambda_argument_name, lambda_argument_type); - } - - size_t previous_scope_node_actions_stack_index = actions_stack.size() - 1; - - auto lambda_actions_dag = std::make_shared(); - actions_stack.emplace_back(lambda_actions_dag, node); - - auto [lambda_expression_node_name, level] = visitImpl(lambda_node.getExpression()); - lambda_actions_dag->getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name)); - lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name)); - - auto lambda_actions = std::make_shared( - lambda_actions_dag, ExpressionActionsSettings::fromContext(planner_context.query_context, CompileExpressions::yes)); - - Names captured_column_names; - ActionsDAG::NodeRawConstPtrs lambda_children; - Names required_column_names = lambda_actions->getRequiredColumns(); - - const auto & lambda_argument_names = lambda_node.getArgumentNames(); - - for (const auto & required_column_name : required_column_names) - { - auto it = std::find_if( - lambda_argument_names.begin(), lambda_argument_names.end(), [&](auto & value) { return value == required_column_name; }); - - if (it == lambda_argument_names.end()) - { - lambda_children.push_back(actions_stack[previous_scope_node_actions_stack_index].getNodeOrThrow(required_column_name)); - captured_column_names.push_back(required_column_name); - } - } - - auto lambda_node_name = getActionsDAGNodeName(node.get()); - auto function_capture = std::make_shared( - lambda_actions, captured_column_names, lambda_arguments_names_and_types, result_type, lambda_expression_node_name); - actions_stack.pop_back(); - - if (level == actions_stack.size()) - --level; - - actions_stack[level].addFunctionIfNecessary(lambda_node_name, lambda_children, function_capture); - - size_t actions_stack_size = actions_stack.size(); - for (size_t i = level + 1; i < actions_stack_size; ++i) - { - auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputColumnIfNecessary(lambda_node_name, result_type); - } - - return {lambda_node_name, level}; - } - - NodeNameAndNodeMinLevel visitFunction(const QueryTreeNodePtr & node) - { - auto function_node_name = getActionsDAGNodeName(node.get()); - const auto & function_node = node->as(); - - if (function_node.getFunctionName() == "grouping") - { - size_t arguments_size = function_node.getArguments().getNodes().size(); - - if (arguments_size == 0) - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function GROUPING expects at least one argument"); - else if (arguments_size > 64) - throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Function GROUPING can have up to 64 arguments, but {} provided", arguments_size); - - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function GROUPING is not supported"); - } - else if (isNameOfInFunction(function_node.getFunctionName())) - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function IN is not supported"); - } - - if (function_node.isAggregateFunction()) - { - size_t actions_stack_size = actions_stack.size(); - - for (size_t i = 0; i < actions_stack_size; ++i) - { - auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputColumnIfNecessary(function_node_name, function_node.getResultType()); - } - - return {function_node_name, 0}; - } - - const auto & function_arguments = function_node.getArguments().getNodes(); - size_t function_arguments_size = function_arguments.size(); - - Names function_arguments_node_names; - function_arguments_node_names.reserve(function_arguments_size); - - size_t level = 0; - for (const auto & argument : function_arguments) - { - if (argument->getNodeType() == QueryTreeNodeType::LAMBDA) - { - auto [node_name, node_min_level] = visitLambda(argument); - function_arguments_node_names.push_back(std::move(node_name)); - level = std::max(level, node_min_level); - continue; - } - - auto [node_name, node_min_level] = visitImpl(argument); - function_arguments_node_names.push_back(std::move(node_name)); - level = std::max(level, node_min_level); - } - - ActionsDAG::NodeRawConstPtrs children; - children.reserve(function_arguments_size); - - for (auto & function_argument_node_name : function_arguments_node_names) - children.push_back(actions_stack[level].getNodeOrThrow(function_argument_node_name)); - - if (function_node.getFunctionName() == "arrayJoin") - { - if (level != 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Expression in arrayJoin cannot depend on lambda argument: {} ", - function_arguments_node_names.at(0)); - - actions_stack[level].addArrayJoinIfNecessary(function_node_name, children.at(0)); - } - else - { - actions_stack[level].addFunctionIfNecessary(function_node_name, children, function_node.getFunction()); - } - - size_t actions_stack_size = actions_stack.size(); - for (size_t i = level + 1; i < actions_stack_size; ++i) - { - auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputColumnIfNecessary(function_node_name, function_node.getResultType()); - } - - return {function_node_name, level}; - } - - String getActionsDAGNodeName(const IQueryTreeNode * node) const - { - String result; - auto node_type = node->getNodeType(); - - switch (node_type) - { - case QueryTreeNodeType::COLUMN: - { - auto it = planner_context.column_node_to_column_identifier.find(node); - if (it == planner_context.column_node_to_column_identifier.end()) - result = node->getName(); - else - result = it->second; - - break; - } - case QueryTreeNodeType::CONSTANT: - { - result = "__constant_" + node->getName(); - break; - } - case QueryTreeNodeType::FUNCTION: - { - const auto & function_node = node->as(); - - WriteBufferFromOwnString buffer; - buffer << "__function_" + function_node.getFunctionName(); - - const auto & function_parameters_nodes = function_node.getParameters().getNodes(); - - if (!function_parameters_nodes.empty()) - { - buffer << '('; - - size_t function_parameters_nodes_size = function_parameters_nodes.size(); - for (size_t i = 0; i < function_parameters_nodes_size; ++i) - { - const auto & function_parameter_node = function_parameters_nodes[i]; - getActionsDAGNodeName(function_parameter_node.get()); - - if (i + 1 != function_parameters_nodes_size) - buffer << ", "; - } - - buffer << ')'; - } - - const auto & function_arguments_nodes = function_node.getArguments().getNodes(); - - buffer << '('; - - size_t function_arguments_nodes_size = function_arguments_nodes.size(); - for (size_t i = 0; i < function_arguments_nodes_size; ++i) - { - const auto & function_argument_node = function_arguments_nodes[i]; - buffer << getActionsDAGNodeName(function_argument_node.get()); - - if (i + 1 != function_arguments_nodes_size) - buffer << ", "; - } - - buffer << ')'; - - result = buffer.str(); - break; - } - case QueryTreeNodeType::QUERY: - { - auto query_hash = node->getTreeHash(); - - result = "__subquery_" + std::to_string(query_hash.first) + '_' + std::to_string(query_hash.second); - break; - } - case QueryTreeNodeType::LAMBDA: - { - auto lambda_hash = node->getTreeHash(); - - result = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); - break; - } - default: - { - result = node->getName(); - break; - } - } - - return result; - } - - std::vector actions_stack; - const PlannerContext & planner_context; -}; - -class CollectTableExpressionIdentifiersVisitor -{ -public: - void visit(const QueryTreeNodePtr & join_tree_node, PlannerContext & planner_context) - { - auto join_tree_node_type = join_tree_node->getNodeType(); - - switch (join_tree_node_type) - { - case QueryTreeNodeType::QUERY: - [[fallthrough]]; - case QueryTreeNodeType::UNION: - [[fallthrough]]; - case QueryTreeNodeType::TABLE: - [[fallthrough]]; - case QueryTreeNodeType::TABLE_FUNCTION: - { - std::string table_expression_identifier = std::to_string(planner_context.table_expression_node_to_identifier.size()); - planner_context.table_expression_node_to_identifier.emplace(join_tree_node.get(), table_expression_identifier); - break; - } - case QueryTreeNodeType::JOIN: - { - auto & join_node = join_tree_node->as(); - visit(join_node.getLeftTableExpression(), planner_context); - - std::string table_expression_identifier = std::to_string(planner_context.table_expression_node_to_identifier.size()); - planner_context.table_expression_node_to_identifier.emplace(join_tree_node.get(), table_expression_identifier); - - visit(join_node.getRightTableExpression(), planner_context); - break; - } - case QueryTreeNodeType::ARRAY_JOIN: - { - auto & array_join_node = join_tree_node->as(); - visit(array_join_node.getTableExpression(), planner_context); - - std::string table_expression_identifier = std::to_string(planner_context.table_expression_node_to_identifier.size()); - planner_context.table_expression_node_to_identifier.emplace(join_tree_node.get(), table_expression_identifier); - break; - } - default: - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected query, table, table function, join or array join query node. Actual {}", - join_tree_node->formatASTForErrorMessage()); - } - } - } -}; - -class CollectSourceColumnsMatcher -{ -public: - using Visitor = InDepthQueryTreeVisitor; - - struct Data - { - PlannerContext & planner_context; - }; - - static void visit(QueryTreeNodePtr & node, Data & data) - { - auto * column_node = node->as(); - if (!column_node) - return; - - auto column_source_node = column_node->getColumnSource(); - auto column_source_node_type = column_source_node->getNodeType(); - - if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN || - column_source_node_type == QueryTreeNodeType::LAMBDA) - return; - - /// JOIN using expression - if (column_node->hasExpression() && column_source_node->getNodeType() == QueryTreeNodeType::JOIN) - return; - - auto & table_expression_node_to_columns = data.planner_context.table_expression_node_to_columns; - auto & table_expression_column_node_to_column_identifier = data.planner_context.column_node_to_column_identifier; - - auto [it, _] = table_expression_node_to_columns.emplace(column_source_node.get(), TableExpressionColumns()); - auto & table_expression_columns = it->second; - - if (column_node->hasExpression()) - { - /// Replace ALIAS column with expression - table_expression_columns.alias_columns.insert(column_node->getColumnName()); - node = column_node->getExpression(); - visit(node, data); - return; - } - - if (column_source_node_type != QueryTreeNodeType::TABLE && - column_source_node_type != QueryTreeNodeType::TABLE_FUNCTION && - column_source_node_type != QueryTreeNodeType::QUERY && - column_source_node_type != QueryTreeNodeType::UNION) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected table, table function, query or union column source. Actual {}", - column_source_node->formatASTForErrorMessage()); - - auto [source_columns_set_it, inserted] = it->second.source_columns_names.insert(column_node->getColumnName()); - - if (inserted) - { - auto column_identifier = data.planner_context.getColumnUniqueIdentifier(column_source_node.get(), column_node->getColumnName()); - table_expression_column_node_to_column_identifier.emplace(column_node, column_identifier); - it->second.column_name_to_column_identifier.emplace(column_node->getColumnName(), column_identifier); - it->second.source_columns.emplace_back(column_node->getColumn()); - } - else - { - auto column_identifier_it = it->second.column_name_to_column_identifier.find(column_node->getColumnName()); - if (column_identifier_it == it->second.column_name_to_column_identifier.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Column node {} column identifier is not initialized", - column_node->formatASTForErrorMessage()); - - table_expression_column_node_to_column_identifier.emplace(column_node, column_identifier_it->second); - } - } - - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) - { - return child_node->getNodeType() != QueryTreeNodeType::QUERY; - } -}; - -using CollectSourceColumnsVisitor = CollectSourceColumnsMatcher::Visitor; - -ActionsDAGPtr convertExpressionNodeIntoDAG(const QueryTreeNodePtr & expression_node, const ColumnsWithTypeAndName & inputs, const PlannerContext & planner_context) -{ - ActionsDAGPtr action_dag = std::make_shared(inputs); - QueryTreeActionsVisitor actions_visitor(action_dag, planner_context); - auto expression_dag_index_nodes = actions_visitor.visit(expression_node); - action_dag->getOutputs().clear(); - - for (auto & expression_dag_index_node : expression_dag_index_nodes) - action_dag->getOutputs().push_back(expression_dag_index_node); - - return action_dag; -} - -QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, - SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, - PlannerContext & planner_context); - -QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, - SelectQueryInfo & table_expression_query_info, - const SelectQueryOptions & select_query_options, - PlannerContext & planner_context) -{ - auto * table_node = table_expression->as(); - auto * table_function_node = table_expression->as(); - auto * query_node = table_expression->as(); - auto * union_node = table_expression->as(); - - QueryPlan query_plan; - - /** Use default columns to support case when there are no columns in query. - * Example: SELECT 1; - */ - const auto & [it, _] = planner_context.table_expression_node_to_columns.emplace(table_expression.get(), TableExpressionColumns()); - auto & table_expression_columns = it->second; - - if (table_node || table_function_node) - { - const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); - const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); - - auto from_stage = storage->getQueryProcessingStage(planner_context.query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); - - Names column_names(table_expression_columns.source_columns_names.begin(), table_expression_columns.source_columns_names.end()); - - std::optional read_additional_column; - - bool plan_has_multiple_table_expressions = planner_context.table_expression_node_to_columns.size() > 1; - if (column_names.empty() && (plan_has_multiple_table_expressions || storage->getName() == "SystemOne")) - { - auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); - read_additional_column = column_names_and_types.front(); - } - - if (read_additional_column) - { - auto column_identifier = planner_context.getColumnUniqueIdentifier(table_expression.get(), read_additional_column->name); - column_names.push_back(read_additional_column->name); - table_expression_columns.source_columns_names.emplace(read_additional_column->name); - table_expression_columns.source_columns.emplace_back(*read_additional_column); - table_expression_columns.column_name_to_column_identifier.emplace(read_additional_column->name, column_identifier); - } - - if (!column_names.empty()) - { - size_t max_block_size = planner_context.query_context->getSettingsRef().max_block_size; - size_t max_streams = planner_context.query_context->getSettingsRef().max_threads; - storage->read(query_plan, column_names, storage_snapshot, table_expression_query_info, planner_context.query_context, from_stage, max_block_size, max_streams); - } - - /// Create step which reads from empty source if storage has no data. - if (!query_plan.isInitialized()) - { - auto source_header = storage_snapshot->getSampleBlockForColumns(column_names); - Pipe pipe(std::make_shared(source_header)); - auto read_from_pipe = std::make_unique(std::move(pipe)); - read_from_pipe->setStepDescription("Read from NullSource"); - query_plan.addStep(std::move(read_from_pipe)); - } - } - else if (query_node || union_node) - { - InterpreterSelectQueryAnalyzer interpeter(table_expression, select_query_options, planner_context.query_context); - interpeter.initializeQueryPlanIfNeeded(); - query_plan = std::move(interpeter).extractQueryPlan(); + return subquery->children[0]; } else { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected table, table function, query or union. Actual {}", table_expression->formatASTForErrorMessage()); - } - - auto rename_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); - - for (const auto & [column_name, column_identifier] : table_expression_columns.column_name_to_column_identifier) - { - auto position = query_plan.getCurrentDataStream().header.getPositionByName(column_name); - const auto * node_to_rename = rename_actions_dag->getOutputs()[position]; - rename_actions_dag->getOutputs()[position] = &rename_actions_dag->addAlias(*node_to_rename, column_identifier); - } - - auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), rename_actions_dag); - rename_step->setStepDescription("Change column names to column identifiers"); - query_plan.addStep(std::move(rename_step)); - - return query_plan; -} - -class JoinClause -{ -public: - void addKey(const ActionsDAG::Node * left_key_node, const ActionsDAG::Node * right_key_node) - { - left_key_nodes.emplace_back(left_key_node); - right_key_nodes.emplace_back(right_key_node); - } - - void addCondition(JoinTableSide table_side, const ActionsDAG::Node * condition_node) - { - auto & filter_condition_nodes = table_side == JoinTableSide::Left ? left_filter_condition_nodes : right_filter_condition_nodes; - filter_condition_nodes.push_back(condition_node); - } - - const ActionsDAG::NodeRawConstPtrs & getLeftKeyNodes() const - { - return left_key_nodes; - } - - const ActionsDAG::NodeRawConstPtrs & getRightKeyNodes() const - { - return right_key_nodes; - } - - ActionsDAG::NodeRawConstPtrs & getLeftKeyNodes() - { - return left_key_nodes; - } - - ActionsDAG::NodeRawConstPtrs & getRightKeyNodes() - { - return right_key_nodes; - } - - const ActionsDAG::NodeRawConstPtrs & getLeftFilterConditionNodes() const - { - return left_filter_condition_nodes; - } - - const ActionsDAG::NodeRawConstPtrs & getRightFilterConditionNodes() const - { - return right_filter_condition_nodes; - } - - void clearConditionNodes(JoinTableSide table_side) - { - auto & filter_condition_nodes = table_side == JoinTableSide::Left ? left_filter_condition_nodes : right_filter_condition_nodes; - filter_condition_nodes.clear(); - } - - void dump(WriteBuffer & buffer) const - { - auto dump_dag_nodes = [&](const ActionsDAG::NodeRawConstPtrs & dag_nodes) - { - String dag_nodes_dump; - - if (!dag_nodes.empty()) - { - for (const auto & dag_node : dag_nodes) - { - dag_nodes_dump += dag_node->result_name; - dag_nodes_dump += ", "; - } - - dag_nodes_dump.pop_back(); - dag_nodes_dump.pop_back(); - } - - return dag_nodes_dump; - }; - - buffer << "left_key_nodes: " << dump_dag_nodes(left_key_nodes); - buffer << " right_key_nodes: " << dump_dag_nodes(right_key_nodes); - - if (!left_filter_condition_nodes.empty()) - buffer << " left_condition_nodes: " + dump_dag_nodes(left_filter_condition_nodes); - - if (!right_filter_condition_nodes.empty()) - buffer << " left_condition_nodes: " + dump_dag_nodes(right_filter_condition_nodes); - } - - [[maybe_unused]] String dump() const - { - WriteBufferFromOwnString buffer; - dump(buffer); - - return buffer.str(); - } -private: - ActionsDAG::NodeRawConstPtrs left_key_nodes; - ActionsDAG::NodeRawConstPtrs right_key_nodes; - - ActionsDAG::NodeRawConstPtrs left_filter_condition_nodes; - ActionsDAG::NodeRawConstPtrs right_filter_condition_nodes; -}; - -using JoinClauses = std::vector; - -std::optional extractJoinTableSideFromExpression(const ActionsDAG::Node * expression_root_node, - const NameSet & left_table_expression_columns_names, - const NameSet & right_table_expression_columns_names, - const JoinNode & join_node) -{ - std::optional table_side; - std::vector nodes_to_process; - nodes_to_process.push_back(expression_root_node); - - while (!nodes_to_process.empty()) - { - const auto * node_to_process = nodes_to_process.back(); - nodes_to_process.pop_back(); - - for (const auto & child : node_to_process->children) - nodes_to_process.push_back(child); - - if (node_to_process->type != ActionsDAG::ActionType::INPUT) - continue; - - const auto & input_name = node_to_process->result_name; - - bool left_table_expression_contains_input = left_table_expression_columns_names.contains(input_name); - bool right_table_expression_contains_input = right_table_expression_columns_names.contains(input_name); - - if (!left_table_expression_contains_input && !right_table_expression_contains_input) - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} actions has column {} that do not exist in left {} or right {} table expression columns", - join_node.formatASTForErrorMessage(), - input_name, - boost::join(left_table_expression_columns_names, ", "), - boost::join(right_table_expression_columns_names, ", ")); - - auto input_table_side = left_table_expression_contains_input ? JoinTableSide::Left : JoinTableSide::Right; - if (table_side && (*table_side) != input_table_side) - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} join expression contains column from left and right table", - join_node.formatASTForErrorMessage()); - - table_side = input_table_side; - } - - return table_side; -} - -void buildJoinClause(ActionsDAGPtr join_expression_dag, - const ActionsDAG::Node * join_expressions_actions_node, - const NameSet & left_table_expression_columns_names, - const NameSet & right_table_expression_columns_names, - const JoinNode & join_node, - JoinClause & join_clause) -{ - /// For and function go into children - if (join_expressions_actions_node->function && join_expressions_actions_node->function->getName() == "and") - { - for (const auto & child : join_expressions_actions_node->children) - { - buildJoinClause(join_expression_dag, - child, - left_table_expression_columns_names, - right_table_expression_columns_names, - join_node, - join_clause); - } - - return; - } - - if (join_expressions_actions_node->function && join_expressions_actions_node->function->getName() == "equals") - { - const auto * equals_left_child = join_expressions_actions_node->children.at(0); - const auto * equals_right_child = join_expressions_actions_node->children.at(1); - - auto left_equals_expression_side_optional = extractJoinTableSideFromExpression(equals_left_child, - left_table_expression_columns_names, - right_table_expression_columns_names, - join_node); - - auto right_equals_expression_side_optional = extractJoinTableSideFromExpression(equals_right_child, - left_table_expression_columns_names, - right_table_expression_columns_names, - join_node); - - if (!left_equals_expression_side_optional && !right_equals_expression_side_optional) - { - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} ON expression {} with constants is not supported", - join_node.formatASTForErrorMessage(), - join_expressions_actions_node->function->getName()); - } - else if (left_equals_expression_side_optional && !right_equals_expression_side_optional) - { - join_clause.addCondition(*left_equals_expression_side_optional, join_expressions_actions_node); - } - else if (!left_equals_expression_side_optional && right_equals_expression_side_optional) - { - join_clause.addCondition(*right_equals_expression_side_optional, join_expressions_actions_node); - } - else - { - auto left_equals_expression_side = *left_equals_expression_side_optional; - auto right_equals_expression_side = *right_equals_expression_side_optional; - - if (left_equals_expression_side != right_equals_expression_side) - { - const ActionsDAG::Node * left_key = equals_left_child; - const ActionsDAG::Node * right_key = equals_right_child; - - if (left_equals_expression_side == JoinTableSide::Right) - { - left_key = equals_right_child; - right_key = equals_left_child; - } - - join_clause.addKey(left_key, right_key); - } - else - { - join_clause.addCondition(left_equals_expression_side, join_expressions_actions_node); - } - } - - return; - } - - auto expression_side_optional = extractJoinTableSideFromExpression(join_expressions_actions_node, - left_table_expression_columns_names, - right_table_expression_columns_names, - join_node); - - if (!expression_side_optional) - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} with constants is not supported", - join_node.formatASTForErrorMessage()); - - auto expression_side = *expression_side_optional; - - join_clause.addCondition(expression_side, join_expressions_actions_node); -} - -struct JoinClausesAndActions -{ - JoinClauses join_clauses; - ActionsDAGPtr join_expression_actions; - ActionsDAGPtr left_join_expressions_actions; - ActionsDAGPtr right_join_expressions_actions; -}; - -JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & join_expression_input_columns, - const ColumnsWithTypeAndName & left_table_expression_columns, - const ColumnsWithTypeAndName & right_table_expression_columns, - const JoinNode & join_node, - const PlannerContext & planner_context) -{ - ActionsDAGPtr join_expression_actions = std::make_shared(join_expression_input_columns); - - QueryTreeActionsVisitor join_expression_visitor(join_expression_actions, planner_context); - auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_node.getJoinExpression()); - if (join_expression_dag_node_raw_pointers.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "JOIN {} ON clause contains multiple expressions", - join_node.formatASTForErrorMessage()); - - const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0]; - if (!join_expressions_actions_root_node->function) - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} join expression expected function", - join_node.formatASTForErrorMessage()); - - size_t left_table_expression_columns_size = left_table_expression_columns.size(); - - Names join_left_actions_names; - join_left_actions_names.reserve(left_table_expression_columns_size); - - NameSet join_left_actions_names_set; - join_left_actions_names_set.reserve(left_table_expression_columns_size); - - for (const auto & left_table_expression_column : left_table_expression_columns) - { - join_left_actions_names.push_back(left_table_expression_column.name); - join_left_actions_names_set.insert(left_table_expression_column.name); - } - - size_t right_table_expression_columns_size = right_table_expression_columns.size(); - - Names join_right_actions_names; - join_right_actions_names.reserve(right_table_expression_columns_size); - - NameSet join_right_actions_names_set; - join_right_actions_names_set.reserve(right_table_expression_columns_size); - - for (const auto & right_table_expression_column : right_table_expression_columns) - { - join_right_actions_names.push_back(right_table_expression_column.name); - join_right_actions_names_set.insert(right_table_expression_column.name); - } - - JoinClausesAndActions result; - result.join_expression_actions = join_expression_actions; - - const auto & function_name = join_expressions_actions_root_node->function->getName(); - if (function_name == "or") - { - for (const auto & child : join_expressions_actions_root_node->children) - { - result.join_clauses.emplace_back(); - - buildJoinClause(join_expression_actions, - child, - join_left_actions_names_set, - join_right_actions_names_set, - join_node, - result.join_clauses.back()); - } - } - else - { - result.join_clauses.emplace_back(); - - buildJoinClause(join_expression_actions, - join_expressions_actions_root_node, - join_left_actions_names_set, - join_right_actions_names_set, - join_node, - result.join_clauses.back()); - } - - auto and_function = FunctionFactory::instance().get("and", planner_context.query_context); - - auto add_necessary_name_if_needed = [&](JoinTableSide join_table_side, const String & name) - { - auto & necessary_names = join_table_side == JoinTableSide::Left ? join_left_actions_names : join_right_actions_names; - auto & necessary_names_set = join_table_side == JoinTableSide::Left ? join_left_actions_names_set : join_right_actions_names_set; - - auto [_, inserted] = necessary_names_set.emplace(name); - if (inserted) - necessary_names.push_back(name); - }; - - for (auto & join_clause : result.join_clauses) - { - const auto & left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); - if (!left_filter_condition_nodes.empty()) - { - const ActionsDAG::Node * dag_filter_condition_node = nullptr; - - if (left_filter_condition_nodes.size() > 1) - dag_filter_condition_node = &join_expression_actions->addFunction(and_function, left_filter_condition_nodes, {}); - else - dag_filter_condition_node = left_filter_condition_nodes[0]; - - join_clause.clearConditionNodes(JoinTableSide::Left); - join_clause.addCondition(JoinTableSide::Left, dag_filter_condition_node); - - join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node); - - add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name); - } - - const auto & right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); - if (!right_filter_condition_nodes.empty()) - { - const ActionsDAG::Node * dag_filter_condition_node = nullptr; - - if (right_filter_condition_nodes.size() > 1) - dag_filter_condition_node = &join_expression_actions->addFunction(and_function, right_filter_condition_nodes, {}); - else - dag_filter_condition_node = right_filter_condition_nodes[0]; - - join_clause.clearConditionNodes(JoinTableSide::Right); - join_clause.addCondition(JoinTableSide::Right, dag_filter_condition_node); - - join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node); - - add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name); - } - - assert(join_clause.getLeftKeyNodes().size() == join_clause.getRightKeyNodes().size()); - size_t join_clause_left_key_nodes_size = join_clause.getLeftKeyNodes().size(); - - for (size_t i = 0; i < join_clause_left_key_nodes_size; ++i) - { - auto & left_key_node = join_clause.getLeftKeyNodes()[i]; - auto & right_key_node = join_clause.getRightKeyNodes()[i]; - - if (!left_key_node->result_type->equals(*right_key_node->result_type)) - { - DataTypePtr common_type; - - try - { - common_type = getLeastSupertype(DataTypes{left_key_node->result_type, right_key_node->result_type}); - } - catch (Exception & ex) - { - ex.addMessage("JOIN {} cannot infer common type in ON section for keys. Left key {} type {}. Right key {} type {}", - join_node.formatASTForErrorMessage(), - left_key_node->result_name, - left_key_node->result_type->getName(), - right_key_node->result_name, - right_key_node->result_type->getName()); - } - - ColumnWithTypeAndName cast_column; - cast_column.name = "__constant_" + common_type->getName(); - cast_column.column = DataTypeString().createColumnConst(0, common_type->getName()); - cast_column.type = std::make_shared(); - - const ActionsDAG::Node * cast_type_constant_node = nullptr; - - if (!left_key_node->result_type->equals(*common_type)) - { - cast_type_constant_node = &join_expression_actions->addColumn(cast_column); - - FunctionCastBase::Diagnostic diagnostic = {left_key_node->result_name, left_key_node->result_name}; - FunctionOverloadResolverPtr func_builder_cast - = CastInternalOverloadResolver::createImpl(diagnostic); - - ActionsDAG::NodeRawConstPtrs children = {left_key_node, cast_type_constant_node}; - left_key_node = &join_expression_actions->addFunction(func_builder_cast, std::move(children), {}); - } - - if (!right_key_node->result_type->equals(*common_type)) - { - if (!cast_type_constant_node) - cast_type_constant_node = &join_expression_actions->addColumn(cast_column); - - FunctionCastBase::Diagnostic diagnostic = {right_key_node->result_name, right_key_node->result_name}; - FunctionOverloadResolverPtr func_builder_cast - = CastInternalOverloadResolver::createImpl(std::move(diagnostic)); - - ActionsDAG::NodeRawConstPtrs children = {right_key_node, cast_type_constant_node}; - right_key_node = &join_expression_actions->addFunction(func_builder_cast, std::move(children), {}); - } - } - - join_expression_actions->addOrReplaceInOutputs(*left_key_node); - join_expression_actions->addOrReplaceInOutputs(*right_key_node); - - add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name); - add_necessary_name_if_needed(JoinTableSide::Right, right_key_node->result_name); - } - } - - result.left_join_expressions_actions = join_expression_actions->clone(); - result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names); - - result.right_join_expressions_actions = join_expression_actions->clone(); - result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); - - return result; -} - -QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, - SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, - PlannerContext & planner_context) -{ - auto & join_node = join_tree_node->as(); - - auto left_plan = buildQueryPlanForJoinTreeNode(join_node.getLeftTableExpression(), - select_query_info, - select_query_options, - planner_context); - auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - - auto right_plan = buildQueryPlanForJoinTreeNode(join_node.getRightTableExpression(), - select_query_info, - select_query_options, - planner_context); - auto right_plan_output_columns = right_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - - if (join_node.getStrictness() == JoinStrictness::Asof) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "JOIN {} ASOF is not supported", - join_node.formatASTForErrorMessage()); - - JoinClausesAndActions join_clauses_and_actions; - - if (join_node.isOnJoinExpression()) - { - auto join_expression_input_columns = left_plan_output_columns; - join_expression_input_columns.insert(join_expression_input_columns.end(), right_plan_output_columns.begin(), right_plan_output_columns.end()); - - join_clauses_and_actions = buildJoinClausesAndActions(join_expression_input_columns, - left_plan_output_columns, - right_plan_output_columns, - join_node, - planner_context); - - auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); - left_join_expressions_actions_step->setStepDescription("JOIN actions"); - left_plan.addStep(std::move(left_join_expressions_actions_step)); - - auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); - right_join_expressions_actions_step->setStepDescription("JOIN actions"); - right_plan.addStep(std::move(right_join_expressions_actions_step)); + "Expected ASTSelectWithUnionQuery or ASTSelectQuery. Actual {}", + query->formatForErrorMessage()); } - - std::unordered_map left_plan_column_name_to_cast_type; - std::unordered_map right_plan_column_name_to_cast_type; - - if (join_node.isUsingJoinExpression()) - { - auto & join_node_using_columns_list = join_node.getJoinExpression()->as(); - for (auto & join_node_using_node : join_node_using_columns_list.getNodes()) - { - auto & join_node_using_column_node = join_node_using_node->as(); - auto & inner_columns_list = join_node_using_column_node.getExpressionOrThrow()->as(); - - auto & left_inner_column_node = inner_columns_list.getNodes().at(0); - auto & left_inner_column = left_inner_column_node->as(); - - auto & right_inner_column_node = inner_columns_list.getNodes().at(1); - auto & right_inner_column = right_inner_column_node->as(); - - const auto & join_node_using_column_node_type = join_node_using_column_node.getColumnType(); - if (!left_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) - { - auto left_inner_column_identifier = planner_context.getColumnIdentifierOrThrow(left_inner_column_node.get()); - left_plan_column_name_to_cast_type.emplace(left_inner_column_identifier, join_node_using_column_node_type); - } - - if (!right_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) - { - auto right_inner_column_identifier = planner_context.getColumnIdentifierOrThrow(right_inner_column_node.get()); - right_plan_column_name_to_cast_type.emplace(right_inner_column_identifier, join_node_using_column_node_type); - } - } - } - - auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map & plan_column_name_to_cast_type) - { - auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); - - for (auto & output_node : cast_actions_dag->getOutputs()) - { - auto it = plan_column_name_to_cast_type.find(output_node->result_name); - if (it == plan_column_name_to_cast_type.end()) - continue; - - const auto & cast_type = it->second; - auto cast_type_name = cast_type->getName(); - - ColumnWithTypeAndName column; - column.name = "__constant_" + cast_type_name; - column.column = DataTypeString().createColumnConst(0, cast_type_name); - column.type = std::make_shared(); - - const auto * cast_type_constant_node = &cast_actions_dag->addColumn(std::move(column)); - - FunctionCastBase::Diagnostic diagnostic = {output_node->result_name, output_node->result_name}; - FunctionOverloadResolverPtr func_builder_cast - = CastInternalOverloadResolver::createImpl(std::move(diagnostic)); - - ActionsDAG::NodeRawConstPtrs children = {output_node, cast_type_constant_node}; - output_node = &cast_actions_dag->addFunction(func_builder_cast, std::move(children), output_node->result_name); - } - - auto cast_join_columns_step - = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); - cast_join_columns_step->setStepDescription("Cast JOIN USING columns"); - plan_to_add_cast.addStep(std::move(cast_join_columns_step)); - }; - - if (!left_plan_column_name_to_cast_type.empty()) - join_cast_plan_output_nodes(left_plan, left_plan_column_name_to_cast_type); - - if (!right_plan_column_name_to_cast_type.empty()) - join_cast_plan_output_nodes(right_plan, right_plan_column_name_to_cast_type); - - JoinKind join_kind = join_node.getKind(); - bool join_use_nulls = planner_context.query_context->getSettingsRef().join_use_nulls; - auto to_nullable_function = FunctionFactory::instance().get("toNullable", planner_context.query_context); - - auto join_cast_plan_columns_to_nullable = [&](QueryPlan & plan_to_add_cast) - { - auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); - - for (auto & output_node : cast_actions_dag->getOutputs()) - { - if (output_node->type == ActionsDAG::ActionType::INPUT && output_node->result_name.starts_with("__column")) - output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); - } - - auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); - cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable"); - plan_to_add_cast.addStep(std::move(cast_join_columns_step)); - }; - - if (join_use_nulls) - { - if (isFull(join_kind)) - { - join_cast_plan_columns_to_nullable(left_plan); - join_cast_plan_columns_to_nullable(right_plan); - } - else if (isLeft(join_kind)) - { - join_cast_plan_columns_to_nullable(right_plan); - } - else if (isRight(join_kind)) - { - join_cast_plan_columns_to_nullable(left_plan); - } - } - - auto table_join = std::make_shared(); - table_join->getTableJoin() = join_node.toASTTableJoin()->as(); - if (join_node.getKind() == JoinKind::Comma) - table_join->getTableJoin().kind = JoinKind::Cross; - table_join->getTableJoin().strictness = JoinStrictness::All; - - if (join_node.isOnJoinExpression()) - { - const auto & join_clauses = join_clauses_and_actions.join_clauses; - auto & table_join_clauses = table_join->getClauses(); - - for (const auto & join_clause : join_clauses) - { - table_join_clauses.emplace_back(); - auto & table_join_clause = table_join_clauses.back(); - - const auto & join_clause_left_key_nodes = join_clause.getLeftKeyNodes(); - const auto & join_clause_right_key_nodes = join_clause.getRightKeyNodes(); - - size_t join_clause_key_nodes_size = join_clause_left_key_nodes.size(); - assert(join_clause_key_nodes_size == join_clause_right_key_nodes.size()); - - for (size_t i = 0; i < join_clause_key_nodes_size; ++i) - { - table_join_clause.key_names_left.push_back(join_clause_left_key_nodes[i]->result_name); - table_join_clause.key_names_right.push_back(join_clause_right_key_nodes[i]->result_name); - } - - const auto & join_clause_get_left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); - if (!join_clause_get_left_filter_condition_nodes.empty()) - { - if (join_clause_get_left_filter_condition_nodes.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "JOIN {} left filter conditions size must be 1. Actual {}", - join_node.formatASTForErrorMessage(), - join_clause_get_left_filter_condition_nodes.size()); - - const auto & join_clause_left_filter_condition_name = join_clause_get_left_filter_condition_nodes[0]->result_name; - table_join_clause.analyzer_left_filter_condition_column_name = join_clause_left_filter_condition_name; - } - - const auto & join_clause_get_right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); - if (!join_clause_get_right_filter_condition_nodes.empty()) - { - if (join_clause_get_right_filter_condition_nodes.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "JOIN {} right filter conditions size must be 1. Actual {}", - join_node.formatASTForErrorMessage(), - join_clause_get_right_filter_condition_nodes.size()); - - const auto & join_clause_right_filter_condition_name = join_clause_get_right_filter_condition_nodes[0]->result_name; - table_join_clause.analyzer_right_filter_condition_column_name = join_clause_right_filter_condition_name; - } - } - } - else if (join_node.isUsingJoinExpression()) - { - auto & table_join_clauses = table_join->getClauses(); - table_join_clauses.emplace_back(); - auto & table_join_clause = table_join_clauses.back(); - - auto & using_list = join_node.getJoinExpression()->as(); - - for (auto & join_using_node : using_list.getNodes()) - { - auto & join_using_column_node = join_using_node->as(); - if (!join_using_column_node.getExpression() || - join_using_column_node.getExpression()->getNodeType() != QueryTreeNodeType::LIST) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "JOIN {} column in USING does not have inner columns", - join_node.formatASTForErrorMessage()); - - auto & using_join_columns_list = join_using_column_node.getExpression()->as(); - auto & using_join_left_join_column_node = using_join_columns_list.getNodes().at(0); - auto & using_join_right_join_column_node = using_join_columns_list.getNodes().at(1); - - auto left_column_identifier_it = planner_context.column_node_to_column_identifier.find(using_join_left_join_column_node.get()); - auto right_column_identifier_it = planner_context.column_node_to_column_identifier.find(using_join_right_join_column_node.get()); - - table_join_clause.key_names_left.push_back(left_column_identifier_it->second); - table_join_clause.key_names_right.push_back(right_column_identifier_it->second); - } - } - - auto left_table_names = left_plan.getCurrentDataStream().header.getNames(); - NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); - - auto columns_from_joined_table = right_plan.getCurrentDataStream().header.getNamesAndTypesList(); - table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); - - for (auto & column_from_joined_table : columns_from_joined_table) - { - if (column_from_joined_table.name.starts_with("__column")) - table_join->addJoinedColumn(column_from_joined_table); - } - - size_t max_block_size = planner_context.query_context->getSettingsRef().max_block_size; - size_t max_streams = planner_context.query_context->getSettingsRef().max_threads; - - JoinPtr join_ptr = std::make_shared(table_join, right_plan.getCurrentDataStream().header, false /*any_take_last_row*/); - QueryPlanStepPtr join_step = std::make_unique( - left_plan.getCurrentDataStream(), - right_plan.getCurrentDataStream(), - join_ptr, - max_block_size, - max_streams, - false /*optimize_read_in_order*/); - - join_step->setStepDescription(fmt::format("JOIN {}", JoinPipelineType::FillRightFirst)); - - std::vector plans; - plans.emplace_back(std::make_unique(std::move(left_plan))); - plans.emplace_back(std::make_unique(std::move(right_plan))); - - auto result_plan = QueryPlan(); - result_plan.unitePlans(std::move(join_step), {std::move(plans)}); - - return result_plan; } -QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression, - SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, - PlannerContext & planner_context) +QueryTreeNodePtr buildQueryTreeAndRunPasses(const ASTPtr & query, const ContextPtr & context) { - auto & array_join_node = table_expression->as(); + auto query_tree = buildQueryTree(query, context); - auto left_plan = buildQueryPlanForJoinTreeNode(array_join_node.getTableExpression(), - select_query_info, - select_query_options, - planner_context); - auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + QueryTreePassManager query_tree_pass_manager(context); + addQueryTreePasses(query_tree_pass_manager); + query_tree_pass_manager.run(query_tree); - ActionsDAGPtr array_join_action_dag = std::make_shared(left_plan_output_columns); - QueryTreeActionsVisitor actions_visitor(array_join_action_dag, planner_context); - - NameSet array_join_columns; - for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) - { - auto & array_join_expression_column = array_join_expression->as(); - const auto & array_join_column_name = array_join_expression_column.getColumnName(); - array_join_columns.insert(array_join_column_name); - - auto expression_dag_index_nodes = actions_visitor.visit(array_join_expression_column.getExpressionOrThrow()); - for (auto & expression_dag_index_node : expression_dag_index_nodes) - { - const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_name); - array_join_action_dag->getOutputs().push_back(array_join_column_node); - } - } - - auto array_join_actions = std::make_unique(left_plan.getCurrentDataStream(), array_join_action_dag); - array_join_actions->setStepDescription("ARRAY JOIN actions"); - left_plan.addStep(std::move(array_join_actions)); - - auto array_join_action = std::make_shared(array_join_columns, array_join_node.isLeft(), planner_context.query_context); - auto array_join_step = std::make_unique(left_plan.getCurrentDataStream(), std::move(array_join_action)); - array_join_step->setStepDescription("ARRAY JOIN"); - left_plan.addStep(std::move(array_join_step)); - - return left_plan; -} - -QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, - SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, - PlannerContext & planner_context) -{ - auto join_tree_node_type = join_tree_node->getNodeType(); - - switch (join_tree_node_type) - { - case QueryTreeNodeType::QUERY: - [[fallthrough]]; - case QueryTreeNodeType::UNION: - [[fallthrough]]; - case QueryTreeNodeType::TABLE: - [[fallthrough]]; - case QueryTreeNodeType::TABLE_FUNCTION: - { - SelectQueryInfo table_expression_query_info = select_query_info; - return buildQueryPlanForTableExpression(join_tree_node, table_expression_query_info, select_query_options, planner_context); - } - case QueryTreeNodeType::JOIN: - { - return buildQueryPlanForJoinNode(join_tree_node, select_query_info, select_query_options, planner_context); - } - case QueryTreeNodeType::ARRAY_JOIN: - { - return buildQueryPlanForArrayJoinNode(join_tree_node, select_query_info, select_query_options, planner_context); - } - default: - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected query, table, table function, join or array join query node. Actual {}", - join_tree_node->formatASTForErrorMessage()); - } - } + return query_tree; } } @@ -2049,56 +108,29 @@ InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( const SelectQueryOptions & select_query_options_, ContextPtr context_) : WithContext(context_) - , query(query_) + , query(normalizeAndValidateQuery(query_)) + , query_tree(buildQueryTreeAndRunPasses(query, context_)) , select_query_options(select_query_options_) + , planner(query_tree, select_query_options, context_) { - if (query->as() || query->as()) - { - } - else if (auto * subquery = query->as()) - { - query = subquery->children[0]; - } - else - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Expected ASTSelectWithUnionQuery or ASTSelectQuery. Actual {}", - query->formatForErrorMessage()); - } - - query_tree = buildQueryTree(query, context_); - - QueryTreePassManager query_tree_pass_manager(context_); - addQueryTreePasses(query_tree_pass_manager); - query_tree_pass_manager.run(query_tree); -} - -InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( - const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_, - ContextPtr context_) - : WithContext(context_) - , query(query_tree_->toAST()) - , query_tree(query_tree_) - , select_query_options(select_query_options_) -{ - if (query_tree->getNodeType() != QueryTreeNodeType::QUERY && - query_tree->getNodeType() != QueryTreeNodeType::UNION) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Expected QUERY or UNION node. Actual {}", - query_tree->formatASTForErrorMessage()); - } Block InterpreterSelectQueryAnalyzer::getSampleBlock() { - initializeQueryPlanIfNeeded(); - return query_plan.getCurrentDataStream().header; + planner.initializeQueryPlanIfNeeded(); + return planner.getQueryPlan().getCurrentDataStream().header; +} + +QueryPlan && InterpreterSelectQueryAnalyzer::extractQueryPlan() && +{ + planner.initializeQueryPlanIfNeeded(); + return std::move(planner).extractQueryPlan(); } BlockIO InterpreterSelectQueryAnalyzer::execute() { - initializeQueryPlanIfNeeded(); + planner.initializeQueryPlanIfNeeded(); + auto & query_plan = planner.getQueryPlan(); QueryPlanOptimizationSettings optimization_settings; BuildQueryPipelineSettings build_pipeline_settings; @@ -2110,195 +142,4 @@ BlockIO InterpreterSelectQueryAnalyzer::execute() return res; } -void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() -{ - if (query_plan.isInitialized()) - return; - - auto current_context = getContext(); - - if (auto * union_query_tree = query_tree->as()) - { - auto union_mode = union_query_tree->getUnionMode(); - if (union_mode == SelectUnionMode::Unspecified) - throw Exception(ErrorCodes::LOGICAL_ERROR, "UNION mode must be initialized"); - - std::vector> query_plans; - Blocks query_plans_headers; - - for (auto & query_node : union_query_tree->getQueries().getNodes()) - { - InterpreterSelectQueryAnalyzer interpeter(query_node, select_query_options, current_context); - interpeter.initializeQueryPlanIfNeeded(); - auto query_node_plan = std::make_unique(std::move(interpeter).extractQueryPlan()); - query_plans_headers.push_back(query_node_plan->getCurrentDataStream().header); - query_plans.push_back(std::move(query_node_plan)); - } - - Block union_common_header = getCommonHeaderForUnion(query_plans_headers); - DataStreams query_plans_streams; - query_plans_streams.reserve(query_plans.size()); - - for (auto & query_node_plan : query_plans) - { - if (blocksHaveEqualStructure(query_node_plan->getCurrentDataStream().header, union_common_header)) - continue; - - auto actions_dag = ActionsDAG::makeConvertingActions( - query_node_plan->getCurrentDataStream().header.getColumnsWithTypeAndName(), - union_common_header.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Position); - auto converting_step = std::make_unique(query_node_plan->getCurrentDataStream(), std::move(actions_dag)); - converting_step->setStepDescription("Conversion before UNION"); - query_node_plan->addStep(std::move(converting_step)); - - query_plans_streams.push_back(query_node_plan->getCurrentDataStream()); - } - - const auto & settings = current_context->getSettingsRef(); - auto max_threads = settings.max_threads; - - if (union_mode == SelectUnionMode::ALL || union_mode == SelectUnionMode::DISTINCT) - { - auto union_step = std::make_unique(std::move(query_plans_streams), max_threads); - query_plan.unitePlans(std::move(union_step), std::move(query_plans)); - - if (union_query_tree->getUnionMode() == SelectUnionMode::DISTINCT) - { - /// Add distinct transform - SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); - - auto distinct_step = std::make_unique( - query_plan.getCurrentDataStream(), - limits, - 0 /*limit hint*/, - query_plan.getCurrentDataStream().header.getNames(), - false /*pre distinct*/, - settings.optimize_distinct_in_order); - - query_plan.addStep(std::move(distinct_step)); - } - } - else if (union_mode == SelectUnionMode::INTERSECT || union_mode == SelectUnionMode::EXCEPT) - { - IntersectOrExceptStep::Operator intersect_or_except_operator = IntersectOrExceptStep::Operator::INTERSECT; - if (union_mode == SelectUnionMode::EXCEPT) - intersect_or_except_operator = IntersectOrExceptStep::Operator::EXCEPT; - - auto union_step = std::make_unique(std::move(query_plans_streams), intersect_or_except_operator, max_threads); - query_plan.unitePlans(std::move(union_step), std::move(query_plans)); - } - - return; - } - - auto & query_node = query_tree->as(); - - SelectQueryInfo select_query_info; - select_query_info.original_query = query; - select_query_info.query = query; - - PlannerContext planner_context; - planner_context.query_context = getContext(); - - CollectTableExpressionIdentifiersVisitor collect_table_expression_identifiers_visitor; - collect_table_expression_identifiers_visitor.visit(query_node.getJoinTree(), planner_context); - - CollectSourceColumnsVisitor::Data data {planner_context}; - CollectSourceColumnsVisitor collect_source_columns_visitor(data); - collect_source_columns_visitor.visit(query_tree); - - query_plan = buildQueryPlanForJoinTreeNode(query_node.getJoinTree(), select_query_info, select_query_options, planner_context); - std::optional> action_chain_node_parent_indices; - - if (query_node.hasWhere()) - { - ColumnsWithTypeAndName where_input; - if (action_chain_node_parent_indices) - planner_context.actions_chain.getAvailableOutputColumns(*action_chain_node_parent_indices); - else - where_input = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - - planner_context.where_actions = convertExpressionNodeIntoDAG(query_node.getWhere(), where_input, planner_context); - planner_context.where_action_node_name = planner_context.where_actions->getOutputs().at(0)->result_name; - - auto where_actions_node = std::make_unique(planner_context.where_actions); - if (action_chain_node_parent_indices) - where_actions_node->addParentIndices(*action_chain_node_parent_indices); - - planner_context.actions_chain.addNode(std::move(where_actions_node)); - action_chain_node_parent_indices = {planner_context.actions_chain.getLastNodeIndex()}; - planner_context.where_actions_chain_node_index = planner_context.actions_chain.size(); - } - - ColumnsWithTypeAndName projection_input; - if (action_chain_node_parent_indices) - planner_context.actions_chain.getAvailableOutputColumns(*action_chain_node_parent_indices); - else - projection_input = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - - planner_context.projection_actions = convertExpressionNodeIntoDAG(query_node.getProjectionNode(), projection_input, planner_context); - - auto projection_actions_node = std::make_unique(planner_context.projection_actions); - if (action_chain_node_parent_indices) - projection_actions_node->addParentIndices(*action_chain_node_parent_indices); - planner_context.actions_chain.addNode(std::move(projection_actions_node)); - - const auto & projection_action_dag_nodes = planner_context.projection_actions->getOutputs(); - size_t projection_action_dag_nodes_size = projection_action_dag_nodes.size(); - - auto & projection_nodes = query_node.getProjection().getNodes(); - size_t projection_nodes_size = projection_nodes.size(); - - if (projection_nodes_size != projection_action_dag_nodes_size) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "QueryTree projection nodes size mismatch. Expected {}. Actual {}", - projection_action_dag_nodes_size, - projection_nodes_size); - - NamesWithAliases projection_names; - - for (size_t i = 0; i < projection_nodes_size; ++i) - { - auto & node = projection_nodes[i]; - auto node_name = node->getName(); - const auto * action_dag_node = projection_action_dag_nodes[i]; - const auto & actions_dag_node_name = action_dag_node->result_name; - - if (node->hasAlias()) - projection_names.push_back({actions_dag_node_name, node->getAlias()}); - else - projection_names.push_back({actions_dag_node_name, node_name}); - } - - planner_context.projection_actions->project(projection_names); - - // std::cout << "Chain dump before finalize" << std::endl; - // std::cout << planner_context.actions_chain.dump() << std::endl; - - planner_context.actions_chain.finalize(); - - // std::cout << "Chain dump after finalize" << std::endl; - // std::cout << planner_context.actions_chain.dump() << std::endl; - - if (query_node.hasWhere()) - { - auto & where_actions_chain_node = planner_context.actions_chain.at(planner_context.where_actions_chain_node_index); - bool remove_filter = !where_actions_chain_node->getChildRequiredOutputColumnsNames().contains(planner_context.where_action_node_name); - auto where_step = std::make_unique(query_plan.getCurrentDataStream(), - planner_context.where_actions, - planner_context.where_action_node_name, - remove_filter); - where_step->setStepDescription("WHERE"); - query_plan.addStep(std::move(where_step)); - } - - // std::cout << "Query plan dump" << std::endl; - // std::cout << dumpQueryPlan(query_plan) << std::endl; - - auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), planner_context.projection_actions); - projection_step->setStepDescription("Projection"); - query_plan.addStep(std::move(projection_step)); -} - } diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 4ba01c26bdd..2f4b472537b 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -7,6 +7,8 @@ #include #include +#include + namespace DB { @@ -14,35 +16,23 @@ class InterpreterSelectQueryAnalyzer : public IInterpreter, public WithContext { public: /// Initialize interpreter with query AST - InterpreterSelectQueryAnalyzer( - const ASTPtr & query_, - const SelectQueryOptions & select_query_options_, - ContextPtr context_); - - /// Initialize interpreter with query tree after query analysis and others phases - InterpreterSelectQueryAnalyzer( - const QueryTreeNodePtr & query_tree_, + InterpreterSelectQueryAnalyzer(const ASTPtr & query_, const SelectQueryOptions & select_query_options_, ContextPtr context_); Block getSampleBlock(); + QueryPlan && extractQueryPlan() &&; + BlockIO execute() override; bool supportsTransactions() const override { return true; } - void initializeQueryPlanIfNeeded(); - - QueryPlan && extractQueryPlan() && - { - return std::move(query_plan); - } - private: ASTPtr query; QueryTreeNodePtr query_tree; - QueryPlan query_plan; SelectQueryOptions select_query_options; + Planner planner; }; } diff --git a/src/Planner/ActionsChain.cpp b/src/Planner/ActionsChain.cpp new file mode 100644 index 00000000000..97f2119b2d7 --- /dev/null +++ b/src/Planner/ActionsChain.cpp @@ -0,0 +1,124 @@ +#include + +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +void ActionsChainStep::finalizeInputAndOutputColumns(const NameSet & child_input_columns) +{ + child_required_output_columns_names.clear(); + std::vector required_output_nodes; + + auto child_input_columns_copy = child_input_columns; + + for (const auto & node : actions->getNodes()) + { + auto it = child_input_columns.find(node.result_name); + + if (it == child_input_columns.end()) + continue; + + child_required_output_columns_names.insert(node.result_name); + required_output_nodes.push_back(&node); + child_input_columns_copy.erase(it); + } + + for (auto & required_output_node : required_output_nodes) + actions->addOrReplaceInOutputs(*required_output_node); + + actions->removeUnusedActions(); + initialize(); +} + +void ActionsChainStep::dump(WriteBuffer & buffer) const +{ + buffer << "DAG" << '\n'; + buffer << actions->dumpDAG(); + if (!child_required_output_columns_names.empty()) + { + buffer << "Child required output columns " << boost::join(child_required_output_columns_names, ", "); + buffer << '\n'; + } +} + +String ActionsChainStep::dump() const +{ + WriteBufferFromOwnString buffer; + dump(buffer); + + return buffer.str(); +} + +void ActionsChainStep::initialize() +{ + auto required_columns_names = actions->getRequiredColumnsNames(); + input_columns_names = NameSet(required_columns_names.begin(), required_columns_names.end()); + + available_output_columns.clear(); + + for (const auto & node : actions->getNodes()) + { + if (available_output_columns_only_aliases) + { + if (node.type == ActionsDAG::ActionType::ALIAS) + available_output_columns.emplace_back(node.column, node.result_type, node.result_name); + + continue; + } + + if (node.type == ActionsDAG::ActionType::INPUT || + node.type == ActionsDAG::ActionType::FUNCTION || + node.type == ActionsDAG::ActionType::ALIAS || + node.type == ActionsDAG::ActionType::ARRAY_JOIN) + available_output_columns.emplace_back(node.column, node.result_type, node.result_name); + } +} + +void ActionsChain::finalize() +{ + if (steps.empty()) + return; + + /// For last chain step there are no columns required in child nodes + NameSet empty_child_input_columns; + steps.back().get()->finalizeInputAndOutputColumns(empty_child_input_columns); + + Int64 steps_last_index = steps.size() - 1; + for (Int64 i = steps_last_index; i >= 1; --i) + { + auto & current_step = steps[i]; + auto & previous_step = steps[i - 1]; + + previous_step->finalizeInputAndOutputColumns(current_step->getInputColumnNames()); + } +} + +void ActionsChain::dump(WriteBuffer & buffer) const +{ + size_t nodes_size = steps.size(); + + for (size_t i = 0; i < nodes_size; ++i) + { + const auto & node = steps[i]; + buffer << "Step " << i << '\n'; + node->dump(buffer); + + buffer << '\n'; + } +} + +String ActionsChain::dump() const +{ + WriteBufferFromOwnString buffer; + dump(buffer); + return buffer.str(); +} + +} diff --git a/src/Planner/ActionsChain.h b/src/Planner/ActionsChain.h new file mode 100644 index 00000000000..c98e8b52e17 --- /dev/null +++ b/src/Planner/ActionsChain.h @@ -0,0 +1,231 @@ +#pragma once + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +/** Chain of query actions steps. This class is needed to eliminate unnecessary actions calculations. + * Each step is represented by actions DAG. + * + * Consider such example query: + * SELECT expr(id) FROM test_table WHERE expr(id) > 0. + * + * We want to reuse expr(id) from previous expressions step, and not recalculate it in projection. + * To do this we build a chain of all query action steps. + * For example: + * 1. before where + * 2. before order by + * 3. projection + * + * Initially root of chain is initialized with join tree query plan header. + * Each next chain step, must be initialized with previous step available output columns. + * That way we forward all available output columns (functions, columns, aliases) from first step of the chain to the + * last step. After chain is build we can finalize it. + * + * Each step has input columns (some of them are not necessary) and output columns. Before chain finalize output columns + * contain only necessary actions for step output calculation. + * For each step starting from last (i), we add columns that are necessary for this step to previous step (i - 1), + * and remove unused input columns of previous step(i - 1). + * That way we reuse already calculated expressions from first step to last step. + */ + +class ActionsChainStep; +using ActionsChainStepPtr = std::unique_ptr; +using ActionsChainSteps = std::vector; + +/// Actions chain step represent single step in actions chain. +class ActionsChainStep +{ +public: + /** Initialize actions step with actions dag. + * Input column names initialized using actions dag nodes with INPUT type. + * Avaiable output columns initialized using actions dag output nodes with INPUT, FUNCTION, ALIAS, ARRAY_JOIN types. + * If available_output_columns_only_aliases is set to true, then only aliases from actions dag will be used + * as available output columns. + */ + explicit ActionsChainStep(ActionsDAGPtr actions_, bool available_output_columns_only_aliases_ = false) + : actions(std::move(actions_)) + , available_output_columns_only_aliases(available_output_columns_only_aliases_) + { + initialize(); + } + + /// Get actions + ActionsDAGPtr & getActions() + { + return actions; + } + + /// Get actions + const ActionsDAGPtr & getActions() const + { + return actions; + } + + /// Get available output columns + const ColumnsWithTypeAndName & getAvailableOutputColumns() const + { + return available_output_columns; + } + + /// Get input column names + const NameSet & getInputColumnNames() const + { + return input_columns_names; + } + + /** Get child required output columns names. + * Initialized during finalizeOutputColumns method call. + */ + const NameSet & getChildRequiredOutputColumnsNames() const + { + return child_required_output_columns_names; + } + + /** Finalize step output columns and remove unnecessary input columns. + * If actions dag node has same name as child input column, it is added to actions output nodes. + */ + void finalizeInputAndOutputColumns(const NameSet & child_input_columns); + + /// Dump step into buffer + void dump(WriteBuffer & buffer) const; + + /// Dump step + String dump() const; + +private: + void initialize(); + + ActionsDAGPtr actions; + + bool available_output_columns_only_aliases; + + NameSet input_columns_names; + + NameSet child_required_output_columns_names; + + ColumnsWithTypeAndName available_output_columns; +}; + +/// Query actions chain +class ActionsChain +{ +public: + /// Add step into actions chain + void addStep(ActionsChainStepPtr step) + { + steps.emplace_back(std::move(step)); + } + + /// Get steps + const ActionsChainSteps & getSteps() const + { + return steps; + } + + /// Get steps size + size_t getStepsSize() const + { + return steps.size(); + } + + const ActionsChainStepPtr & at(size_t index) const + { + if (index >= steps.size()) + throw std::out_of_range("actions chain access is out of range"); + + return steps[index]; + } + + ActionsChainStepPtr & at(size_t index) + { + if (index >= steps.size()) + throw std::out_of_range("actions chain access is out of range"); + + return steps[index]; + } + + ActionsChainStepPtr & operator[](size_t index) + { + return steps[index]; + } + + const ActionsChainStepPtr & operator[](size_t index) const + { + return steps[index]; + } + + /// Get last step + ActionsChainStep * getLastStep() + { + return steps.back().get(); + } + + /// Get last step or throw exception if chain is empty + ActionsChainStep * getLastStepOrThrow() + { + if (steps.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ActionsChain is empty"); + + return steps.back().get(); + } + + /// Get last step index + size_t getLastStepIndex() + { + return steps.size() - 1; + } + + /// Get last step index or throw exception if chain is empty + size_t getLastStepIndexOrThrow() + { + if (steps.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ActionsChain is empty"); + + return steps.size() - 1; + } + + /// Get last step available output columns + const ColumnsWithTypeAndName & getLastStepAvailableOutputColumns() const + { + return steps.back()->getAvailableOutputColumns(); + } + + /// Get last step available output columns or throw exception if chain is empty + const ColumnsWithTypeAndName & getLastStepAvailableOutputColumnsOrThrow() const + { + if (steps.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ActionsChain is empty"); + + return steps.back()->getAvailableOutputColumns(); + } + + /// Get last step available output columns or throw exception if chain is empty + const ColumnsWithTypeAndName * getLastStepAvailableOutputColumnsOrNull() const + { + if (steps.empty()) + return nullptr; + + return &steps.back()->getAvailableOutputColumns(); + } + + /// Finalize chain + void finalize(); + + /// Dump chain into buffer + void dump(WriteBuffer & buffer) const; + + /// Dump chain + String dump() const; + +private: + ActionsChainSteps steps; +}; + +} diff --git a/src/Planner/CMakeLists.txt b/src/Planner/CMakeLists.txt new file mode 100644 index 00000000000..1068fee4cea --- /dev/null +++ b/src/Planner/CMakeLists.txt @@ -0,0 +1,7 @@ +if (ENABLE_TESTS) + add_subdirectory(tests) +endif() + +if (ENABLE_EXAMPLES) + add_subdirectory(examples) +endif() \ No newline at end of file diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp new file mode 100644 index 00000000000..820af998823 --- /dev/null +++ b/src/Planner/Planner.cpp @@ -0,0 +1,889 @@ +#include + +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; + extern const int LOGICAL_ERROR; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; + extern const int INVALID_JOIN_ON_EXPRESSION; +} + +/** ClickHouse query planner. + * + * TODO: JOIN support ASOF. JOIN support strictness. JOIN support constants. JOIN support ON t1.id = t1.id + * TODO: JOIN drop unnecessary columns after ON, USING section + * TODO: Support display names + * TODO: Support RBAC. Support RBAC for ALIAS columns. + * TODO: Support distributed query processing + * TODO: Support PREWHERE + * TODO: Support GROUP BY, HAVING + * TODO: Support ORDER BY, LIMIT + * TODO: Support WINDOW FUNCTIONS + * TODO: Support DISTINCT + * TODO: Support building sets for IN functions + * TODO: Support trivial count optimization + * TODO: Support totals, extremes + * TODO: Support projections + * TODO: Support read in order optimization + * TODO: UNION storage limits + * TODO: Interpreter resources + */ + +namespace +{ + +class CollectTableExpressionIdentifiersVisitor +{ +public: + void visit(const QueryTreeNodePtr & join_tree_node, PlannerContext & planner_context) + { + auto & table_expression_node_to_identifier = planner_context.getTableExpressionNodeToIdentifier(); + auto join_tree_node_type = join_tree_node->getNodeType(); + + switch (join_tree_node_type) + { + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::UNION: + [[fallthrough]]; + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + std::string table_expression_identifier = std::to_string(table_expression_node_to_identifier.size()); + table_expression_node_to_identifier.emplace(join_tree_node.get(), table_expression_identifier); + break; + } + case QueryTreeNodeType::JOIN: + { + auto & join_node = join_tree_node->as(); + visit(join_node.getLeftTableExpression(), planner_context); + + std::string table_expression_identifier = std::to_string(table_expression_node_to_identifier.size()); + table_expression_node_to_identifier.emplace(join_tree_node.get(), table_expression_identifier); + + visit(join_node.getRightTableExpression(), planner_context); + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + auto & array_join_node = join_tree_node->as(); + visit(array_join_node.getTableExpression(), planner_context); + + std::string table_expression_identifier = std::to_string(table_expression_node_to_identifier.size()); + table_expression_node_to_identifier.emplace(join_tree_node.get(), table_expression_identifier); + break; + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected query, table, table function, join or array join query node. Actual {}", + join_tree_node->formatASTForErrorMessage()); + } + } + } +}; + +class CollectSourceColumnsMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + PlannerContext & planner_context; + }; + + static void visit(QueryTreeNodePtr & node, Data & data) + { + auto * column_node = node->as(); + if (!column_node) + return; + + auto column_source_node = column_node->getColumnSource(); + auto column_source_node_type = column_source_node->getNodeType(); + + if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN || + column_source_node_type == QueryTreeNodeType::LAMBDA) + return; + + /// JOIN using expression + if (column_node->hasExpression() && column_source_node->getNodeType() == QueryTreeNodeType::JOIN) + return; + + auto & table_expression_node_to_columns = data.planner_context.getTableExpressionNodeToColumns(); + + auto [it, _] = table_expression_node_to_columns.emplace(column_source_node.get(), TableExpressionColumns()); + auto & table_expression_columns = it->second; + + if (column_node->hasExpression()) + { + /// Replace ALIAS column with expression + table_expression_columns.addAliasColumnName(column_node->getColumnName()); + node = column_node->getExpression(); + visit(node, data); + return; + } + + if (column_source_node_type != QueryTreeNodeType::TABLE && + column_source_node_type != QueryTreeNodeType::TABLE_FUNCTION && + column_source_node_type != QueryTreeNodeType::QUERY && + column_source_node_type != QueryTreeNodeType::UNION) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected table, table function, query or union column source. Actual {}", + column_source_node->formatASTForErrorMessage()); + + bool column_already_exists = table_expression_columns.hasColumn(column_node->getColumnName()); + + if (!column_already_exists) + { + auto column_identifier = data.planner_context.getColumnUniqueIdentifier(column_source_node.get(), column_node->getColumnName()); + data.planner_context.registerColumnNode(column_node, column_identifier); + table_expression_columns.addColumn(column_node->getColumn(), column_identifier); + } + else + { + auto column_identifier = table_expression_columns.getColumnIdentifierOrThrow(column_node->getColumnName()); + data.planner_context.registerColumnNode(column_node, column_identifier); + } + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) + { + return child_node->getNodeType() != QueryTreeNodeType::QUERY; + } +}; + +using CollectSourceColumnsVisitor = CollectSourceColumnsMatcher::Visitor; + +ActionsDAGPtr convertExpressionNodeIntoDAG(const QueryTreeNodePtr & expression_node, const ColumnsWithTypeAndName & inputs, const PlannerContextPtr & planner_context) +{ + ActionsDAGPtr action_dag = std::make_shared(inputs); + PlannerActionsVisitor actions_visitor(action_dag, planner_context); + auto expression_dag_index_nodes = actions_visitor.visit(expression_node); + action_dag->getOutputs().clear(); + + for (auto & expression_dag_index_node : expression_dag_index_nodes) + action_dag->getOutputs().push_back(expression_dag_index_node); + + return action_dag; +} + +QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContextPtr & planner_context); + +QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, + SelectQueryInfo & table_expression_query_info, + const SelectQueryOptions & select_query_options, + PlannerContextPtr & planner_context) +{ + auto * table_node = table_expression->as(); + auto * table_function_node = table_expression->as(); + auto * query_node = table_expression->as(); + auto * union_node = table_expression->as(); + + QueryPlan query_plan; + + auto & table_expression_node_to_columns = planner_context->getTableExpressionNodeToColumns(); + + /** Use default columns to support case when there are no columns in query. + * Example: SELECT 1; + */ + const auto & [it, _] = table_expression_node_to_columns.emplace(table_expression.get(), TableExpressionColumns()); + auto & table_expression_columns = it->second; + + if (table_node || table_function_node) + { + const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + + auto from_stage = storage->getQueryProcessingStage(planner_context->getQueryContext(), select_query_options.to_stage, storage_snapshot, table_expression_query_info); + const auto & columns_names = table_expression_columns.getColumnsNames(); + Names column_names(columns_names.begin(), columns_names.end()); + + std::optional read_additional_column; + + bool plan_has_multiple_table_expressions = table_expression_node_to_columns.size() > 1; + if (column_names.empty() && (plan_has_multiple_table_expressions || storage->getName() == "SystemOne")) + { + auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); + read_additional_column = column_names_and_types.front(); + } + + if (read_additional_column) + { + auto column_identifier = planner_context->getColumnUniqueIdentifier(table_expression.get(), read_additional_column->name); + column_names.push_back(read_additional_column->name); + table_expression_columns.addColumn(*read_additional_column, column_identifier); + } + + if (!column_names.empty()) + { + const auto & query_context = planner_context->getQueryContext(); + size_t max_block_size = query_context->getSettingsRef().max_block_size; + size_t max_streams = query_context->getSettingsRef().max_threads; + storage->read(query_plan, column_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); + } + + /// Create step which reads from empty source if storage has no data. + if (!query_plan.isInitialized()) + { + auto source_header = storage_snapshot->getSampleBlockForColumns(column_names); + Pipe pipe(std::make_shared(source_header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource"); + query_plan.addStep(std::move(read_from_pipe)); + } + } + else if (query_node || union_node) + { + Planner subquery_planner(table_expression, select_query_options, planner_context->getQueryContext(), planner_context->getGlobalPlannerContext()); + subquery_planner.initializeQueryPlanIfNeeded(); + query_plan = std::move(subquery_planner).extractQueryPlan(); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected table, table function, query or union. Actual {}", table_expression->formatASTForErrorMessage()); + } + + auto rename_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + + for (const auto & [column_name, column_identifier] : table_expression_columns.getColumnNameToIdentifier()) + { + auto position = query_plan.getCurrentDataStream().header.getPositionByName(column_name); + const auto * node_to_rename = rename_actions_dag->getOutputs()[position]; + rename_actions_dag->getOutputs()[position] = &rename_actions_dag->addAlias(*node_to_rename, column_identifier); + } + + auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), rename_actions_dag); + rename_step->setStepDescription("Change column names to column identifiers"); + query_plan.addStep(std::move(rename_step)); + + return query_plan; +} + +QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContextPtr & planner_context) +{ + auto & join_node = join_tree_node->as(); + + auto left_plan = buildQueryPlanForJoinTreeNode(join_node.getLeftTableExpression(), + select_query_info, + select_query_options, + planner_context); + auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + auto right_plan = buildQueryPlanForJoinTreeNode(join_node.getRightTableExpression(), + select_query_info, + select_query_options, + planner_context); + auto right_plan_output_columns = right_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + if (join_node.getStrictness() == JoinStrictness::Asof) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "JOIN {} ASOF is not supported", + join_node.formatASTForErrorMessage()); + + JoinClausesAndActions join_clauses_and_actions; + + if (join_node.isOnJoinExpression()) + { + join_clauses_and_actions = buildJoinClausesAndActions(left_plan_output_columns, + right_plan_output_columns, + join_tree_node, + planner_context); + + auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); + left_join_expressions_actions_step->setStepDescription("JOIN actions"); + left_plan.addStep(std::move(left_join_expressions_actions_step)); + + auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); + right_join_expressions_actions_step->setStepDescription("JOIN actions"); + right_plan.addStep(std::move(right_join_expressions_actions_step)); + } + + std::unordered_map left_plan_column_name_to_cast_type; + std::unordered_map right_plan_column_name_to_cast_type; + + if (join_node.isUsingJoinExpression()) + { + auto & join_node_using_columns_list = join_node.getJoinExpression()->as(); + for (auto & join_node_using_node : join_node_using_columns_list.getNodes()) + { + auto & join_node_using_column_node = join_node_using_node->as(); + auto & inner_columns_list = join_node_using_column_node.getExpressionOrThrow()->as(); + + auto & left_inner_column_node = inner_columns_list.getNodes().at(0); + auto & left_inner_column = left_inner_column_node->as(); + + auto & right_inner_column_node = inner_columns_list.getNodes().at(1); + auto & right_inner_column = right_inner_column_node->as(); + + const auto & join_node_using_column_node_type = join_node_using_column_node.getColumnType(); + if (!left_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) + { + auto left_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(left_inner_column_node.get()); + left_plan_column_name_to_cast_type.emplace(left_inner_column_identifier, join_node_using_column_node_type); + } + + if (!right_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) + { + auto right_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(right_inner_column_node.get()); + right_plan_column_name_to_cast_type.emplace(right_inner_column_identifier, join_node_using_column_node_type); + } + } + } + + auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map & plan_column_name_to_cast_type) + { + auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); + + for (auto & output_node : cast_actions_dag->getOutputs()) + { + auto it = plan_column_name_to_cast_type.find(output_node->result_name); + if (it == plan_column_name_to_cast_type.end()) + continue; + + const auto & cast_type = it->second; + auto cast_type_name = cast_type->getName(); + + ColumnWithTypeAndName column; + column.name = "__constant_" + cast_type_name; + column.column = DataTypeString().createColumnConst(0, cast_type_name); + column.type = std::make_shared(); + + const auto * cast_type_constant_node = &cast_actions_dag->addColumn(std::move(column)); + + FunctionCastBase::Diagnostic diagnostic = {output_node->result_name, output_node->result_name}; + FunctionOverloadResolverPtr func_builder_cast + = CastInternalOverloadResolver::createImpl(std::move(diagnostic)); + + ActionsDAG::NodeRawConstPtrs children = {output_node, cast_type_constant_node}; + output_node = &cast_actions_dag->addFunction(func_builder_cast, std::move(children), output_node->result_name); + } + + auto cast_join_columns_step + = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); + cast_join_columns_step->setStepDescription("Cast JOIN USING columns"); + plan_to_add_cast.addStep(std::move(cast_join_columns_step)); + }; + + if (!left_plan_column_name_to_cast_type.empty()) + join_cast_plan_output_nodes(left_plan, left_plan_column_name_to_cast_type); + + if (!right_plan_column_name_to_cast_type.empty()) + join_cast_plan_output_nodes(right_plan, right_plan_column_name_to_cast_type); + + const auto & query_context = planner_context->getQueryContext(); + JoinKind join_kind = join_node.getKind(); + bool join_use_nulls = query_context->getSettingsRef().join_use_nulls; + auto to_nullable_function = FunctionFactory::instance().get("toNullable", query_context); + + auto join_cast_plan_columns_to_nullable = [&](QueryPlan & plan_to_add_cast) + { + auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); + + for (auto & output_node : cast_actions_dag->getOutputs()) + { + if (output_node->type == ActionsDAG::ActionType::INPUT && output_node->result_name.starts_with("__column")) + output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); + } + + auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); + cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable"); + plan_to_add_cast.addStep(std::move(cast_join_columns_step)); + }; + + if (join_use_nulls) + { + if (isFull(join_kind)) + { + join_cast_plan_columns_to_nullable(left_plan); + join_cast_plan_columns_to_nullable(right_plan); + } + else if (isLeft(join_kind)) + { + join_cast_plan_columns_to_nullable(right_plan); + } + else if (isRight(join_kind)) + { + join_cast_plan_columns_to_nullable(left_plan); + } + } + + auto table_join = std::make_shared(); + table_join->getTableJoin() = join_node.toASTTableJoin()->as(); + if (join_node.getKind() == JoinKind::Comma) + table_join->getTableJoin().kind = JoinKind::Cross; + table_join->getTableJoin().strictness = JoinStrictness::All; + + if (join_node.isOnJoinExpression()) + { + const auto & join_clauses = join_clauses_and_actions.join_clauses; + auto & table_join_clauses = table_join->getClauses(); + + for (const auto & join_clause : join_clauses) + { + table_join_clauses.emplace_back(); + auto & table_join_clause = table_join_clauses.back(); + + const auto & join_clause_left_key_nodes = join_clause.getLeftKeyNodes(); + const auto & join_clause_right_key_nodes = join_clause.getRightKeyNodes(); + + size_t join_clause_key_nodes_size = join_clause_left_key_nodes.size(); + assert(join_clause_key_nodes_size == join_clause_right_key_nodes.size()); + + for (size_t i = 0; i < join_clause_key_nodes_size; ++i) + { + table_join_clause.key_names_left.push_back(join_clause_left_key_nodes[i]->result_name); + table_join_clause.key_names_right.push_back(join_clause_right_key_nodes[i]->result_name); + } + + const auto & join_clause_get_left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); + if (!join_clause_get_left_filter_condition_nodes.empty()) + { + if (join_clause_get_left_filter_condition_nodes.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} left filter conditions size must be 1. Actual {}", + join_node.formatASTForErrorMessage(), + join_clause_get_left_filter_condition_nodes.size()); + + const auto & join_clause_left_filter_condition_name = join_clause_get_left_filter_condition_nodes[0]->result_name; + table_join_clause.analyzer_left_filter_condition_column_name = join_clause_left_filter_condition_name; + } + + const auto & join_clause_get_right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); + if (!join_clause_get_right_filter_condition_nodes.empty()) + { + if (join_clause_get_right_filter_condition_nodes.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} right filter conditions size must be 1. Actual {}", + join_node.formatASTForErrorMessage(), + join_clause_get_right_filter_condition_nodes.size()); + + const auto & join_clause_right_filter_condition_name = join_clause_get_right_filter_condition_nodes[0]->result_name; + table_join_clause.analyzer_right_filter_condition_column_name = join_clause_right_filter_condition_name; + } + } + } + else if (join_node.isUsingJoinExpression()) + { + auto & table_join_clauses = table_join->getClauses(); + table_join_clauses.emplace_back(); + auto & table_join_clause = table_join_clauses.back(); + + auto & using_list = join_node.getJoinExpression()->as(); + + for (auto & join_using_node : using_list.getNodes()) + { + auto & join_using_column_node = join_using_node->as(); + if (!join_using_column_node.getExpression() || + join_using_column_node.getExpression()->getNodeType() != QueryTreeNodeType::LIST) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} column in USING does not have inner columns", + join_node.formatASTForErrorMessage()); + + auto & using_join_columns_list = join_using_column_node.getExpression()->as(); + auto & using_join_left_join_column_node = using_join_columns_list.getNodes().at(0); + auto & using_join_right_join_column_node = using_join_columns_list.getNodes().at(1); + + auto left_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_left_join_column_node.get()); + auto right_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_right_join_column_node.get()); + + table_join_clause.key_names_left.push_back(left_column_identifier); + table_join_clause.key_names_right.push_back(right_column_identifier); + } + } + + auto left_table_names = left_plan.getCurrentDataStream().header.getNames(); + NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); + + auto columns_from_joined_table = right_plan.getCurrentDataStream().header.getNamesAndTypesList(); + table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); + + for (auto & column_from_joined_table : columns_from_joined_table) + { + if (column_from_joined_table.name.starts_with("__column")) + table_join->addJoinedColumn(column_from_joined_table); + } + + size_t max_block_size = query_context->getSettingsRef().max_block_size; + size_t max_streams = query_context->getSettingsRef().max_threads; + + JoinPtr join_ptr = std::make_shared(table_join, right_plan.getCurrentDataStream().header, false /*any_take_last_row*/); + QueryPlanStepPtr join_step = std::make_unique( + left_plan.getCurrentDataStream(), + right_plan.getCurrentDataStream(), + join_ptr, + max_block_size, + max_streams, + false /*optimize_read_in_order*/); + + join_step->setStepDescription(fmt::format("JOIN {}", JoinPipelineType::FillRightFirst)); + + std::vector plans; + plans.emplace_back(std::make_unique(std::move(left_plan))); + plans.emplace_back(std::make_unique(std::move(right_plan))); + + auto result_plan = QueryPlan(); + result_plan.unitePlans(std::move(join_step), {std::move(plans)}); + + return result_plan; +} + +QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContextPtr & planner_context) +{ + auto & array_join_node = table_expression->as(); + + auto left_plan = buildQueryPlanForJoinTreeNode(array_join_node.getTableExpression(), + select_query_info, + select_query_options, + planner_context); + auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + ActionsDAGPtr array_join_action_dag = std::make_shared(left_plan_output_columns); + PlannerActionsVisitor actions_visitor(array_join_action_dag, planner_context); + + NameSet array_join_columns; + for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) + { + auto & array_join_expression_column = array_join_expression->as(); + const auto & array_join_column_name = array_join_expression_column.getColumnName(); + array_join_columns.insert(array_join_column_name); + + auto expression_dag_index_nodes = actions_visitor.visit(array_join_expression_column.getExpressionOrThrow()); + for (auto & expression_dag_index_node : expression_dag_index_nodes) + { + const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_name); + array_join_action_dag->getOutputs().push_back(array_join_column_node); + } + } + + auto array_join_actions = std::make_unique(left_plan.getCurrentDataStream(), array_join_action_dag); + array_join_actions->setStepDescription("ARRAY JOIN actions"); + left_plan.addStep(std::move(array_join_actions)); + + auto array_join_action = std::make_shared(array_join_columns, array_join_node.isLeft(), planner_context->getQueryContext()); + auto array_join_step = std::make_unique(left_plan.getCurrentDataStream(), std::move(array_join_action)); + array_join_step->setStepDescription("ARRAY JOIN"); + left_plan.addStep(std::move(array_join_step)); + + return left_plan; +} + +QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContextPtr & planner_context) +{ + auto join_tree_node_type = join_tree_node->getNodeType(); + + switch (join_tree_node_type) + { + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::UNION: + [[fallthrough]]; + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + SelectQueryInfo table_expression_query_info = select_query_info; + return buildQueryPlanForTableExpression(join_tree_node, table_expression_query_info, select_query_options, planner_context); + } + case QueryTreeNodeType::JOIN: + { + return buildQueryPlanForJoinNode(join_tree_node, select_query_info, select_query_options, planner_context); + } + case QueryTreeNodeType::ARRAY_JOIN: + { + return buildQueryPlanForArrayJoinNode(join_tree_node, select_query_info, select_query_options, planner_context); + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected query, table, table function, join or array join query node. Actual {}", + join_tree_node->formatASTForErrorMessage()); + } + } +} + +} + +Planner::Planner(const QueryTreeNodePtr & query_tree_, + const SelectQueryOptions & select_query_options_, + ContextPtr context_) + : WithContext(context_) + , query_tree(query_tree_) + , select_query_options(select_query_options_) + , planner_context(std::make_shared(context_, std::make_shared())) +{ + if (query_tree->getNodeType() != QueryTreeNodeType::QUERY && + query_tree->getNodeType() != QueryTreeNodeType::UNION) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Expected QUERY or UNION node. Actual {}", + query_tree->formatASTForErrorMessage()); +} + +/// Initialize interpreter with query tree after query analysis phase and global planner context +Planner::Planner(const QueryTreeNodePtr & query_tree_, + const SelectQueryOptions & select_query_options_, + ContextPtr context_, + GlobalPlannerContextPtr global_planner_context_) + : WithContext(context_) + , query_tree(query_tree_) + , select_query_options(select_query_options_) + , planner_context(std::make_shared(context_, std::move(global_planner_context_))) +{ + if (query_tree->getNodeType() != QueryTreeNodeType::QUERY && + query_tree->getNodeType() != QueryTreeNodeType::UNION) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Expected QUERY or UNION node. Actual {}", + query_tree->formatASTForErrorMessage()); +} + +void Planner::initializeQueryPlanIfNeeded() +{ + if (query_plan.isInitialized()) + return; + + auto current_context = getContext(); + + if (auto * union_query_tree = query_tree->as()) + { + auto union_mode = union_query_tree->getUnionMode(); + if (union_mode == SelectUnionMode::Unspecified) + throw Exception(ErrorCodes::LOGICAL_ERROR, "UNION mode must be initialized"); + + size_t queries_size = union_query_tree->getQueries().getNodes().size(); + + std::vector> query_plans; + query_plans.reserve(queries_size); + + Blocks query_plans_headers; + query_plans_headers.reserve(queries_size); + + for (auto & query_node : union_query_tree->getQueries().getNodes()) + { + Planner query_planner(query_node, select_query_options, current_context); + query_planner.initializeQueryPlanIfNeeded(); + auto query_node_plan = std::make_unique(std::move(query_planner).extractQueryPlan()); + query_plans_headers.push_back(query_node_plan->getCurrentDataStream().header); + query_plans.push_back(std::move(query_node_plan)); + } + + Block union_common_header = buildCommonHeaderForUnion(query_plans_headers); + DataStreams query_plans_streams; + query_plans_streams.reserve(query_plans.size()); + + for (auto & query_node_plan : query_plans) + { + if (blocksHaveEqualStructure(query_node_plan->getCurrentDataStream().header, union_common_header)) + continue; + + auto actions_dag = ActionsDAG::makeConvertingActions( + query_node_plan->getCurrentDataStream().header.getColumnsWithTypeAndName(), + union_common_header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + auto converting_step = std::make_unique(query_node_plan->getCurrentDataStream(), std::move(actions_dag)); + converting_step->setStepDescription("Conversion before UNION"); + query_node_plan->addStep(std::move(converting_step)); + + query_plans_streams.push_back(query_node_plan->getCurrentDataStream()); + } + + const auto & settings = current_context->getSettingsRef(); + auto max_threads = settings.max_threads; + + if (union_mode == SelectUnionMode::ALL || union_mode == SelectUnionMode::DISTINCT) + { + auto union_step = std::make_unique(std::move(query_plans_streams), max_threads); + query_plan.unitePlans(std::move(union_step), std::move(query_plans)); + + if (union_query_tree->getUnionMode() == SelectUnionMode::DISTINCT) + { + /// Add distinct transform + SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); + + auto distinct_step = std::make_unique( + query_plan.getCurrentDataStream(), + limits, + 0 /*limit hint*/, + query_plan.getCurrentDataStream().header.getNames(), + false /*pre distinct*/, + settings.optimize_distinct_in_order); + + query_plan.addStep(std::move(distinct_step)); + } + } + else if (union_mode == SelectUnionMode::INTERSECT || union_mode == SelectUnionMode::EXCEPT) + { + IntersectOrExceptStep::Operator intersect_or_except_operator = IntersectOrExceptStep::Operator::INTERSECT; + if (union_mode == SelectUnionMode::EXCEPT) + intersect_or_except_operator = IntersectOrExceptStep::Operator::EXCEPT; + + auto union_step = std::make_unique(std::move(query_plans_streams), intersect_or_except_operator, max_threads); + query_plan.unitePlans(std::move(union_step), std::move(query_plans)); + } + + return; + } + + auto & query_node = query_tree->as(); + auto query = query_node.toAST(); + + SelectQueryInfo select_query_info; + select_query_info.original_query = query; + select_query_info.query = query; + + CollectTableExpressionIdentifiersVisitor collect_table_expression_identifiers_visitor; + collect_table_expression_identifiers_visitor.visit(query_node.getJoinTree(), *planner_context); + + CollectSourceColumnsVisitor::Data data {*planner_context}; + CollectSourceColumnsVisitor collect_source_columns_visitor(data); + collect_source_columns_visitor.visit(query_tree); + + query_plan = buildQueryPlanForJoinTreeNode(query_node.getJoinTree(), select_query_info, select_query_options, planner_context); + + ActionsChain actions_chain; + std::optional where_action_step_index; + std::string where_action_node_name; + + if (query_node.hasWhere()) + { + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & where_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + auto where_actions = convertExpressionNodeIntoDAG(query_node.getWhere(), where_input, planner_context); + where_action_node_name = where_actions->getOutputs().at(0)->result_name; + actions_chain.addStep(std::make_unique(std::move(where_actions))); + where_action_step_index = actions_chain.getLastStepIndex(); + } + + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + auto projection_actions = convertExpressionNodeIntoDAG(query_node.getProjectionNode(), projection_input, planner_context); + + const auto & projection_action_dag_nodes = projection_actions->getOutputs(); + size_t projection_action_dag_nodes_size = projection_action_dag_nodes.size(); + + auto & projection_nodes = query_node.getProjection().getNodes(); + size_t projection_nodes_size = projection_nodes.size(); + + if (projection_nodes_size != projection_action_dag_nodes_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "QueryTree projection nodes size mismatch. Expected {}. Actual {}", + projection_action_dag_nodes_size, + projection_nodes_size); + + NamesWithAliases projection_names; + + for (size_t i = 0; i < projection_nodes_size; ++i) + { + auto & node = projection_nodes[i]; + auto node_name = node->getName(); + const auto * action_dag_node = projection_action_dag_nodes[i]; + const auto & actions_dag_node_name = action_dag_node->result_name; + + if (node->hasAlias()) + projection_names.push_back({actions_dag_node_name, node->getAlias()}); + else + projection_names.push_back({actions_dag_node_name, node_name}); + } + + projection_actions->project(projection_names); + + actions_chain.addStep(std::make_unique(std::move(projection_actions))); + size_t projection_action_step_index = actions_chain.getLastStepIndex(); + + // std::cout << "Chain dump before finalize" << std::endl; + // std::cout << planner_context.actions_chain.dump() << std::endl; + + actions_chain.finalize(); + + // std::cout << "Chain dump after finalize" << std::endl; + // std::cout << planner_context.actions_chain.dump() << std::endl; + + if (where_action_step_index) + { + auto & where_actions_chain_node = actions_chain.at(*where_action_step_index); + bool remove_filter = !where_actions_chain_node->getChildRequiredOutputColumnsNames().contains(where_action_node_name); + auto where_step = std::make_unique(query_plan.getCurrentDataStream(), + where_actions_chain_node->getActions(), + where_action_node_name, + remove_filter); + where_step->setStepDescription("WHERE"); + query_plan.addStep(std::move(where_step)); + } + + // std::cout << "Query plan dump" << std::endl; + // std::cout << dumpQueryPlan(query_plan) << std::endl; + + auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), actions_chain[projection_action_step_index]->getActions()); + projection_step->setStepDescription("Projection"); + query_plan.addStep(std::move(projection_step)); +} + +} diff --git a/src/Planner/Planner.h b/src/Planner/Planner.h new file mode 100644 index 00000000000..19f5f582fcf --- /dev/null +++ b/src/Planner/Planner.h @@ -0,0 +1,57 @@ +#pragma once + +#include +#include + +#include +#include +#include + +namespace DB +{ + +class GlobalPlannerContext; +using GlobalPlannerContextPtr = std::shared_ptr; + +class PlannerContext; +using PlannerContextPtr = std::shared_ptr; + +class Planner : public WithContext +{ +public: + /// Initialize planner with query tree after analysis phase + Planner(const QueryTreeNodePtr & query_tree_, + const SelectQueryOptions & select_query_options_, + ContextPtr context_); + + /// Initialize interpreter with query tree after query analysis phase and global planner context + Planner(const QueryTreeNodePtr & query_tree_, + const SelectQueryOptions & select_query_options_, + ContextPtr context_, + GlobalPlannerContextPtr global_planner_context_); + + const QueryPlan & getQueryPlan() const + { + return query_plan; + } + + QueryPlan & getQueryPlan() + { + return query_plan; + } + + void initializeQueryPlanIfNeeded(); + + QueryPlan && extractQueryPlan() && + { + return std::move(query_plan); + } + +private: + QueryTreeNodePtr query_tree; + QueryPlan query_plan; + SelectQueryOptions select_query_options; + PlannerContextPtr planner_context; +}; + +} diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp new file mode 100644 index 00000000000..cdeb8ef0225 --- /dev/null +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -0,0 +1,466 @@ +#include + +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; + extern const int LOGICAL_ERROR; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; +} + +class PlannerActionsVisitor::ActionsScopeNode +{ +public: + explicit ActionsScopeNode(ActionsDAGPtr actions_dag_, QueryTreeNodePtr scope_node_) + : actions_dag(std::move(actions_dag_)) + , scope_node(std::move(scope_node_)) + { + for (const auto & node : actions_dag->getNodes()) + node_name_to_node[node.result_name] = &node; + } + + const QueryTreeNodePtr & getScopeNode() const + { + return scope_node; + } + + [[maybe_unused]] bool containsNode(const std::string & node_name) + { + return node_name_to_node.find(node_name) != node_name_to_node.end(); + } + + [[maybe_unused]] const ActionsDAG::Node * tryGetNode(const std::string & node_name) + { + auto it = node_name_to_node.find(node_name); + if (it == node_name_to_node.end()) + return {}; + + return it->second; + } + + const ActionsDAG::Node * getNodeOrThrow(const std::string & node_name) + { + auto it = node_name_to_node.find(node_name); + if (it == node_name_to_node.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "No node with name {}. There are only nodes {}", + node_name, + actions_dag->dumpNames()); + + return it->second; + } + + const ActionsDAG::Node * addInputColumnIfNecessary(const std::string & node_name, const DataTypePtr & column_type) + { + auto it = node_name_to_node.find(node_name); + if (it != node_name_to_node.end()) + return it->second; + + const auto * node = &actions_dag->addInput(node_name, column_type); + node_name_to_node[node->result_name] = node; + + return node; + } + + const ActionsDAG::Node * addInputConstantColumnIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) + { + auto it = node_name_to_node.find(node_name); + if (it != node_name_to_node.end()) + return it->second; + + const auto * node = &actions_dag->addInput(column); + node_name_to_node[node->result_name] = node; + + return node; + } + + const ActionsDAG::Node * addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) + { + auto it = node_name_to_node.find(node_name); + if (it != node_name_to_node.end()) + return it->second; + + const auto * node = &actions_dag->addColumn(column); + node_name_to_node[node->result_name] = node; + + return node; + } + + const ActionsDAG::Node * addFunctionIfNecessary(const std::string & node_name, ActionsDAG::NodeRawConstPtrs children, FunctionOverloadResolverPtr function) + { + auto it = node_name_to_node.find(node_name); + if (it != node_name_to_node.end()) + return it->second; + + const auto * node = &actions_dag->addFunction(function, children, node_name); + node_name_to_node[node->result_name] = node; + + return node; + } + + const ActionsDAG::Node * addArrayJoinIfNecessary(const std::string & node_name, const ActionsDAG::Node * child) + { + auto it = node_name_to_node.find(node_name); + if (it != node_name_to_node.end()) + return it->second; + + const auto * node = &actions_dag->addArrayJoin(*child, node_name); + node_name_to_node[node->result_name] = node; + + return node; + } + +private: + std::unordered_map node_name_to_node; + ActionsDAGPtr actions_dag; + QueryTreeNodePtr scope_node; +}; + +PlannerActionsVisitor::PlannerActionsVisitor(ActionsDAGPtr actions_dag, const PlannerContextPtr & planner_context_) + : planner_context(planner_context_) +{ + actions_stack.emplace_back(std::make_unique(std::move(actions_dag), nullptr)); +} + +PlannerActionsVisitor::~PlannerActionsVisitor() = default; + +ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(QueryTreeNodePtr expression_node) +{ + ActionsDAG::NodeRawConstPtrs result; + + if (auto * expression_list_node = expression_node->as()) + { + for (auto & node : expression_list_node->getNodes()) + { + auto [node_name, _] = visitImpl(node); + result.push_back(actions_stack.front()->getNodeOrThrow(node_name)); + } + } + else + { + auto [node_name, _] = visitImpl(expression_node); + result.push_back(actions_stack.front()->getNodeOrThrow(node_name)); + } + + return result; +} + +PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitImpl(QueryTreeNodePtr node) +{ + if (auto * column_node = node->as()) + return visitColumn(node); + else if (auto * constant_node = node->as()) + return visitConstant(node); + else if (auto * function_node = node->as()) + return visitFunction(node); + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Expected only column, constant or function node. Actual {}", + node->formatASTForErrorMessage()); +} + +PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitColumn(const QueryTreeNodePtr & node) +{ + auto column_node_name = getActionsDAGNodeName(node.get()); + const auto & column_node = node->as(); + + Int64 actions_stack_size = static_cast(actions_stack.size() - 1); + for (Int64 i = actions_stack_size; i >= 0; --i) + { + actions_stack[i]->addInputColumnIfNecessary(column_node_name, column_node.getColumnType()); + + if (column_node.getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA + && actions_stack[i]->getScopeNode().get() == column_node.getColumnSource().get()) + { + return {column_node_name, i}; + } + } + + return {column_node_name, 0}; +} + +PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitConstant(const QueryTreeNodePtr & node) +{ + auto constant_node_name = getActionsDAGNodeName(node.get()); + const auto & constant_node = node->as(); + const auto & literal = constant_node.getConstantValue(); + + ColumnWithTypeAndName column; + column.name = constant_node_name; + column.type = constant_node.getResultType(); + column.column = column.type->createColumnConst(1, literal); + + actions_stack[0]->addConstantIfNecessary(constant_node_name, column); + + size_t actions_stack_size = actions_stack.size(); + for (size_t i = 1; i < actions_stack_size; ++i) + { + auto & actions_stack_node = actions_stack[i]; + actions_stack_node->addInputConstantColumnIfNecessary(constant_node_name, column); + } + + return {constant_node_name, 0}; +} + +PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitLambda(const QueryTreeNodePtr & node) +{ + auto & lambda_node = node->as(); + auto result_type = lambda_node.getResultType(); + if (!result_type) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Lambda {} is not resolved during query analysis", + lambda_node.formatASTForErrorMessage()); + + NamesAndTypesList lambda_arguments_names_and_types; + + for (auto & lambda_node_argument : lambda_node.getArguments().getNodes()) + { + auto lambda_argument_name = lambda_node_argument->getName(); + auto lambda_argument_type = lambda_node_argument->getResultType(); + lambda_arguments_names_and_types.emplace_back(lambda_argument_name, lambda_argument_type); + } + + size_t previous_scope_node_actions_stack_index = actions_stack.size() - 1; + + auto lambda_actions_dag = std::make_shared(); + actions_stack.emplace_back(std::make_unique(lambda_actions_dag, node)); + + auto [lambda_expression_node_name, level] = visitImpl(lambda_node.getExpression()); + lambda_actions_dag->getOutputs().push_back(actions_stack.back()->getNodeOrThrow(lambda_expression_node_name)); + lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name)); + + auto expression_actions_settings = ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::yes); + auto lambda_actions = std::make_shared(lambda_actions_dag, expression_actions_settings); + + Names captured_column_names; + ActionsDAG::NodeRawConstPtrs lambda_children; + Names required_column_names = lambda_actions->getRequiredColumns(); + + const auto & lambda_argument_names = lambda_node.getArgumentNames(); + + for (const auto & required_column_name : required_column_names) + { + auto it = std::find_if( + lambda_argument_names.begin(), lambda_argument_names.end(), [&](auto & value) { return value == required_column_name; }); + + if (it == lambda_argument_names.end()) + { + lambda_children.push_back(actions_stack[previous_scope_node_actions_stack_index]->getNodeOrThrow(required_column_name)); + captured_column_names.push_back(required_column_name); + } + } + + auto lambda_node_name = getActionsDAGNodeName(node.get()); + auto function_capture = std::make_shared( + lambda_actions, captured_column_names, lambda_arguments_names_and_types, result_type, lambda_expression_node_name); + actions_stack.pop_back(); + + if (level == actions_stack.size()) + --level; + + actions_stack[level]->addFunctionIfNecessary(lambda_node_name, lambda_children, function_capture); + + size_t actions_stack_size = actions_stack.size(); + for (size_t i = level + 1; i < actions_stack_size; ++i) + { + auto & actions_stack_node = actions_stack[i]; + actions_stack_node->addInputColumnIfNecessary(lambda_node_name, result_type); + } + + return {lambda_node_name, level}; +} + +PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitFunction(const QueryTreeNodePtr & node) +{ + auto function_node_name = getActionsDAGNodeName(node.get()); + const auto & function_node = node->as(); + + if (function_node.getFunctionName() == "grouping") + { + size_t arguments_size = function_node.getArguments().getNodes().size(); + + if (arguments_size == 0) + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, + "Function GROUPING expects at least one argument"); + else if (arguments_size > 64) + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, + "Function GROUPING can have up to 64 arguments, but {} provided", + arguments_size); + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function GROUPING is not supported"); + } + else if (isNameOfInFunction(function_node.getFunctionName())) + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function IN is not supported"); + } + + if (function_node.isAggregateFunction()) + { + size_t actions_stack_size = actions_stack.size(); + + for (size_t i = 0; i < actions_stack_size; ++i) + { + auto & actions_stack_node = actions_stack[i]; + actions_stack_node->addInputColumnIfNecessary(function_node_name, function_node.getResultType()); + } + + return {function_node_name, 0}; + } + + const auto & function_arguments = function_node.getArguments().getNodes(); + size_t function_arguments_size = function_arguments.size(); + + Names function_arguments_node_names; + function_arguments_node_names.reserve(function_arguments_size); + + size_t level = 0; + for (const auto & argument : function_arguments) + { + if (argument->getNodeType() == QueryTreeNodeType::LAMBDA) + { + auto [node_name, node_min_level] = visitLambda(argument); + function_arguments_node_names.push_back(std::move(node_name)); + level = std::max(level, node_min_level); + continue; + } + + auto [node_name, node_min_level] = visitImpl(argument); + function_arguments_node_names.push_back(std::move(node_name)); + level = std::max(level, node_min_level); + } + + ActionsDAG::NodeRawConstPtrs children; + children.reserve(function_arguments_size); + + for (auto & function_argument_node_name : function_arguments_node_names) + children.push_back(actions_stack[level]->getNodeOrThrow(function_argument_node_name)); + + if (function_node.getFunctionName() == "arrayJoin") + { + if (level != 0) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Expression in arrayJoin cannot depend on lambda argument: {} ", + function_arguments_node_names.at(0)); + + actions_stack[level]->addArrayJoinIfNecessary(function_node_name, children.at(0)); + } + else + { + actions_stack[level]->addFunctionIfNecessary(function_node_name, children, function_node.getFunction()); + } + + size_t actions_stack_size = actions_stack.size(); + for (size_t i = level + 1; i < actions_stack_size; ++i) + { + auto & actions_stack_node = actions_stack[i]; + actions_stack_node->addInputColumnIfNecessary(function_node_name, function_node.getResultType()); + } + + return {function_node_name, level}; +} + + String PlannerActionsVisitor::getActionsDAGNodeName(const IQueryTreeNode * node) const + { + String result; + auto node_type = node->getNodeType(); + + switch (node_type) + { + case QueryTreeNodeType::COLUMN: + { + const auto * column_identifier = planner_context->getColumnNodeIdentifierOrNull(node); + result = column_identifier ? *column_identifier : node->getName(); + + break; + } + case QueryTreeNodeType::CONSTANT: + { + std::string constant_value_dump = node->as().getConstantValue().dump(); + result = "__constant_" + constant_value_dump; + break; + } + case QueryTreeNodeType::FUNCTION: + { + const auto & function_node = node->as(); + + WriteBufferFromOwnString buffer; + buffer << "__function_" + function_node.getFunctionName(); + + const auto & function_parameters_nodes = function_node.getParameters().getNodes(); + + if (!function_parameters_nodes.empty()) + { + buffer << '('; + + size_t function_parameters_nodes_size = function_parameters_nodes.size(); + for (size_t i = 0; i < function_parameters_nodes_size; ++i) + { + const auto & function_parameter_node = function_parameters_nodes[i]; + getActionsDAGNodeName(function_parameter_node.get()); + + if (i + 1 != function_parameters_nodes_size) + buffer << ", "; + } + + buffer << ')'; + } + + const auto & function_arguments_nodes = function_node.getArguments().getNodes(); + + buffer << '('; + + size_t function_arguments_nodes_size = function_arguments_nodes.size(); + for (size_t i = 0; i < function_arguments_nodes_size; ++i) + { + const auto & function_argument_node = function_arguments_nodes[i]; + buffer << getActionsDAGNodeName(function_argument_node.get()); + + if (i + 1 != function_arguments_nodes_size) + buffer << ", "; + } + + buffer << ')'; + + result = buffer.str(); + break; + } + case QueryTreeNodeType::QUERY: + { + auto query_hash = node->getTreeHash(); + + result = "__subquery_" + std::to_string(query_hash.first) + '_' + std::to_string(query_hash.second); + break; + } + case QueryTreeNodeType::LAMBDA: + { + auto lambda_hash = node->getTreeHash(); + + result = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); + break; + } + default: + { + result = node->getName(); + break; + } + } + + return result; + } + +} diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h new file mode 100644 index 00000000000..571d8fe8be3 --- /dev/null +++ b/src/Planner/PlannerActionsVisitor.h @@ -0,0 +1,50 @@ +#pragma once + +#include + +#include +#include + +#include +#include + +#include + +#include + +namespace DB +{ + +class PlannerContext; +using PlannerContextPtr = std::shared_ptr; + +class PlannerActionsVisitor +{ +public: + explicit PlannerActionsVisitor(ActionsDAGPtr actions_dag, const PlannerContextPtr & planner_context_); + + ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node); + + ~PlannerActionsVisitor(); + +private: + using NodeNameAndNodeMinLevel = std::pair; + + NodeNameAndNodeMinLevel visitImpl(QueryTreeNodePtr node); + + NodeNameAndNodeMinLevel visitColumn(const QueryTreeNodePtr & node); + + NodeNameAndNodeMinLevel visitConstant(const QueryTreeNodePtr & node); + + NodeNameAndNodeMinLevel visitLambda(const QueryTreeNodePtr & node); + + NodeNameAndNodeMinLevel visitFunction(const QueryTreeNodePtr & node); + + String getActionsDAGNodeName(const IQueryTreeNode * node) const; + + class ActionsScopeNode; + std::vector> actions_stack; + const PlannerContextPtr planner_context; +}; + +} diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp new file mode 100644 index 00000000000..df179cd544f --- /dev/null +++ b/src/Planner/PlannerContext.cpp @@ -0,0 +1,93 @@ +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +PlannerContext::PlannerContext(ContextPtr query_context_, GlobalPlannerContextPtr global_planner_context_) + : query_context(std::move(query_context_)) + , global_planner_context(std::move(global_planner_context_)) +{} + +void PlannerContext::registerColumnNode(const IQueryTreeNode * column_node, const ColumnIdentifier & column_identifier) +{ + assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); + column_node_to_column_identifier.emplace(column_node, column_identifier); +} + +const ColumnIdentifier & PlannerContext::getColumnNodeIdentifierOrThrow(const IQueryTreeNode * column_node) +{ + assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); + + auto it = column_node_to_column_identifier.find(column_node); + if (it == column_node_to_column_identifier.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column identifier is not initialized for column {}", + column_node->formatASTForErrorMessage()); + + return it->second; +} + +const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const IQueryTreeNode * column_node) +{ + assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); + + auto it = column_node_to_column_identifier.find(column_node); + if (it == column_node_to_column_identifier.end()) + return nullptr; + + return &it->second; +} + +ColumnIdentifier PlannerContext::getColumnUniqueIdentifier(const IQueryTreeNode * column_source_node, std::string column_name) +{ + auto column_unique_prefix = "__column_" + std::to_string(column_identifier_counter); + ++column_identifier_counter; + + std::string table_expression_identifier; + auto table_expression_identifier_it = table_expression_node_to_identifier.find(column_source_node); + if (table_expression_identifier_it != table_expression_node_to_identifier.end()) + table_expression_identifier = table_expression_identifier_it->second; + + std::string debug_identifier_suffix; + + if (column_source_node->hasAlias()) + { + debug_identifier_suffix += column_source_node->getAlias(); + } + else if (const auto * table_source_node = column_source_node->as()) + { + debug_identifier_suffix += table_source_node->getStorageID().getFullNameNotQuoted(); + } + else + { + auto column_source_node_type = column_source_node->getNodeType(); + if (column_source_node_type == QueryTreeNodeType::JOIN) + debug_identifier_suffix += "join"; + else if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN) + debug_identifier_suffix += "array_join"; + else if (column_source_node_type == QueryTreeNodeType::TABLE_FUNCTION) + debug_identifier_suffix += "table_function"; + else if (column_source_node_type == QueryTreeNodeType::QUERY) + debug_identifier_suffix += "subquery"; + + if (!table_expression_identifier.empty()) + debug_identifier_suffix += '_' + table_expression_identifier; + } + + if (!column_name.empty()) + debug_identifier_suffix += '.' + column_name; + + if (!debug_identifier_suffix.empty()) + column_unique_prefix += '_' + debug_identifier_suffix; + + return column_unique_prefix; +} + +} diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h new file mode 100644 index 00000000000..fa50bdf66a4 --- /dev/null +++ b/src/Planner/PlannerContext.h @@ -0,0 +1,218 @@ +#pragma once + +#include + +#include +#include + +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +using ColumnIdentifier = std::string; + +class TableExpressionColumns +{ +public: + using ColumnNameToColumnIdentifier = std::unordered_map; + + bool hasColumn(const std::string & column_name) const + { + return alias_columns_names.contains(column_name) || columns_names.contains(column_name); + } + + void addColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) + { + if (hasColumn(column.name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Column with name {} already exists"); + + columns_names.insert(column.name); + columns.push_back(column); + column_name_to_column_identifier.emplace(column.name, column_identifier); + } + + void addColumnIfNotExists(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) + { + if (hasColumn(column.name)) + return; + + columns_names.insert(column.name); + columns.push_back(column); + column_name_to_column_identifier.emplace(column.name, column_identifier); + } + + void addAliasColumnName(const std::string & column_name) + { + alias_columns_names.insert(column_name); + } + + const NameSet & getAliasColumnsNames() const + { + return alias_columns_names; + } + + const NameSet & getColumnsNames() const + { + return columns_names; + } + + const NamesAndTypesList & getColumns() const + { + return columns; + } + + const ColumnNameToColumnIdentifier & getColumnNameToIdentifier() const + { + return column_name_to_column_identifier; + } + + const ColumnIdentifier & getColumnIdentifierOrThrow(const std::string & column_name) const + { + auto it = column_name_to_column_identifier.find(column_name); + if (it == column_name_to_column_identifier.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column identifier for name {} does not exists", + column_name); + + return it->second; + } + + const ColumnIdentifier * getColumnIdentifierOrNull(const std::string & column_name) const + { + auto it = column_name_to_column_identifier.find(column_name); + if (it == column_name_to_column_identifier.end()) + return nullptr; + + return &it->second; + } + +private: + /// Valid for table, table function, query table expression nodes + NamesAndTypesList columns; + + /// Valid for table, table function, query table expression nodes + NameSet columns_names; + + /// Valid only for table table expression node + NameSet alias_columns_names; + + /// Valid for table, table function, query table expression nodes + ColumnNameToColumnIdentifier column_name_to_column_identifier; +}; + +class Set; +using SetPtr = std::shared_ptr; + +class GlobalPlannerContext +{ +public: + GlobalPlannerContext() = default; + + void registerSet(UInt128 source_hash, SetPtr set) + { + set_source_to_set.emplace(source_hash, set); + } + + SetPtr getSet(UInt128 source_hash) const + { + auto it = set_source_to_set.find(source_hash); + if (it == set_source_to_set.end()) + return nullptr; + + return it->second; + } + + void registerSubqueryForSet(String key, SubqueryForSet subquery_for_set) + { + subqueries_for_sets.emplace(key, std::move(subquery_for_set)); + } + + const SubqueriesForSets & getSubqueriesForSets() const + { + return subqueries_for_sets; + } +private: + std::unordered_map set_source_to_set; + + SubqueriesForSets subqueries_for_sets; +}; + +using GlobalPlannerContextPtr = std::shared_ptr; + +class PlannerContext +{ +public: + PlannerContext(ContextPtr query_context_, GlobalPlannerContextPtr global_planner_context_); + + const ContextPtr & getQueryContext() const + { + return query_context; + } + + const GlobalPlannerContextPtr & getGlobalPlannerContext() const + { + return global_planner_context; + } + + GlobalPlannerContextPtr & getGlobalPlannerContext() + { + return global_planner_context; + } + + const std::unordered_map & getTableExpressionNodeToIdentifier() const + { + return table_expression_node_to_identifier; + } + + std::unordered_map & getTableExpressionNodeToIdentifier() + { + return table_expression_node_to_identifier; + } + + const std::unordered_map & getTableExpressionNodeToColumns() const + { + return table_expression_node_to_columns; + } + + std::unordered_map & getTableExpressionNodeToColumns() + { + return table_expression_node_to_columns; + } + + ColumnIdentifier getColumnUniqueIdentifier(const IQueryTreeNode * column_source_node, std::string column_name = {}); + + void registerColumnNode(const IQueryTreeNode * column_node, const ColumnIdentifier & column_identifier); + + const ColumnIdentifier & getColumnNodeIdentifierOrThrow(const IQueryTreeNode * column_node); + + const ColumnIdentifier * getColumnNodeIdentifierOrNull(const IQueryTreeNode * column_node); + +private: + /// Query context + ContextPtr query_context; + + /// Global planner context + GlobalPlannerContextPtr global_planner_context; + + /// Column node to column identifier + std::unordered_map column_node_to_column_identifier; + + /// Table expression to identifier + std::unordered_map table_expression_node_to_identifier; + + /// Table expression node to columns + std::unordered_map table_expression_node_to_columns; + + size_t column_identifier_counter = 0; +}; + +} diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp new file mode 100644 index 00000000000..813a8ecb2c6 --- /dev/null +++ b/src/Planner/PlannerJoins.cpp @@ -0,0 +1,441 @@ +#include + +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include +#include +#include +#include + +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INVALID_JOIN_ON_EXPRESSION; + +} + +void JoinClause::dump(WriteBuffer & buffer) const +{ + auto dump_dag_nodes = [&](const ActionsDAG::NodeRawConstPtrs & dag_nodes) + { + String dag_nodes_dump; + + if (!dag_nodes.empty()) + { + for (const auto & dag_node : dag_nodes) + { + dag_nodes_dump += dag_node->result_name; + dag_nodes_dump += ", "; + } + + dag_nodes_dump.pop_back(); + dag_nodes_dump.pop_back(); + } + + return dag_nodes_dump; + }; + + buffer << "left_key_nodes: " << dump_dag_nodes(left_key_nodes); + buffer << " right_key_nodes: " << dump_dag_nodes(right_key_nodes); + + if (!left_filter_condition_nodes.empty()) + buffer << " left_condition_nodes: " + dump_dag_nodes(left_filter_condition_nodes); + + if (!right_filter_condition_nodes.empty()) + buffer << " left_condition_nodes: " + dump_dag_nodes(right_filter_condition_nodes); +} + +String JoinClause::dump() const +{ + WriteBufferFromOwnString buffer; + dump(buffer); + + return buffer.str(); +} + +namespace +{ + +std::optional extractJoinTableSideFromExpression(const ActionsDAG::Node * expression_root_node, + const NameSet & left_table_expression_columns_names, + const NameSet & right_table_expression_columns_names, + const JoinNode & join_node) +{ + std::optional table_side; + std::vector nodes_to_process; + nodes_to_process.push_back(expression_root_node); + + while (!nodes_to_process.empty()) + { + const auto * node_to_process = nodes_to_process.back(); + nodes_to_process.pop_back(); + + for (const auto & child : node_to_process->children) + nodes_to_process.push_back(child); + + if (node_to_process->type != ActionsDAG::ActionType::INPUT) + continue; + + const auto & input_name = node_to_process->result_name; + + bool left_table_expression_contains_input = left_table_expression_columns_names.contains(input_name); + bool right_table_expression_contains_input = right_table_expression_columns_names.contains(input_name); + + if (!left_table_expression_contains_input && !right_table_expression_contains_input) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} actions has column {} that do not exist in left {} or right {} table expression columns", + join_node.formatASTForErrorMessage(), + input_name, + boost::join(left_table_expression_columns_names, ", "), + boost::join(right_table_expression_columns_names, ", ")); + + auto input_table_side = left_table_expression_contains_input ? JoinTableSide::Left : JoinTableSide::Right; + if (table_side && (*table_side) != input_table_side) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} join expression contains column from left and right table", + join_node.formatASTForErrorMessage()); + + table_side = input_table_side; + } + + return table_side; +} + +void buildJoinClause(ActionsDAGPtr join_expression_dag, + const ActionsDAG::Node * join_expressions_actions_node, + const NameSet & left_table_expression_columns_names, + const NameSet & right_table_expression_columns_names, + const JoinNode & join_node, + JoinClause & join_clause) +{ + /// For and function go into children + if (join_expressions_actions_node->function && join_expressions_actions_node->function->getName() == "and") + { + for (const auto & child : join_expressions_actions_node->children) + { + buildJoinClause(join_expression_dag, + child, + left_table_expression_columns_names, + right_table_expression_columns_names, + join_node, + join_clause); + } + + return; + } + + if (join_expressions_actions_node->function && join_expressions_actions_node->function->getName() == "equals") + { + const auto * equals_left_child = join_expressions_actions_node->children.at(0); + const auto * equals_right_child = join_expressions_actions_node->children.at(1); + + auto left_equals_expression_side_optional = extractJoinTableSideFromExpression(equals_left_child, + left_table_expression_columns_names, + right_table_expression_columns_names, + join_node); + + auto right_equals_expression_side_optional = extractJoinTableSideFromExpression(equals_right_child, + left_table_expression_columns_names, + right_table_expression_columns_names, + join_node); + + if (!left_equals_expression_side_optional && !right_equals_expression_side_optional) + { + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} ON expression {} with constants is not supported", + join_node.formatASTForErrorMessage(), + join_expressions_actions_node->function->getName()); + } + else if (left_equals_expression_side_optional && !right_equals_expression_side_optional) + { + join_clause.addCondition(*left_equals_expression_side_optional, join_expressions_actions_node); + } + else if (!left_equals_expression_side_optional && right_equals_expression_side_optional) + { + join_clause.addCondition(*right_equals_expression_side_optional, join_expressions_actions_node); + } + else + { + auto left_equals_expression_side = *left_equals_expression_side_optional; + auto right_equals_expression_side = *right_equals_expression_side_optional; + + if (left_equals_expression_side != right_equals_expression_side) + { + const ActionsDAG::Node * left_key = equals_left_child; + const ActionsDAG::Node * right_key = equals_right_child; + + if (left_equals_expression_side == JoinTableSide::Right) + { + left_key = equals_right_child; + right_key = equals_left_child; + } + + join_clause.addKey(left_key, right_key); + } + else + { + join_clause.addCondition(left_equals_expression_side, join_expressions_actions_node); + } + } + + return; + } + + auto expression_side_optional = extractJoinTableSideFromExpression(join_expressions_actions_node, + left_table_expression_columns_names, + right_table_expression_columns_names, + join_node); + + if (!expression_side_optional) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} with constants is not supported", + join_node.formatASTForErrorMessage()); + + auto expression_side = *expression_side_optional; + + join_clause.addCondition(expression_side, join_expressions_actions_node); +} + +JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & join_expression_input_columns, + const ColumnsWithTypeAndName & left_table_expression_columns, + const ColumnsWithTypeAndName & right_table_expression_columns, + const JoinNode & join_node, + const PlannerContextPtr & planner_context) +{ + ActionsDAGPtr join_expression_actions = std::make_shared(join_expression_input_columns); + + PlannerActionsVisitor join_expression_visitor(join_expression_actions, planner_context); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_node.getJoinExpression()); + if (join_expression_dag_node_raw_pointers.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} ON clause contains multiple expressions", + join_node.formatASTForErrorMessage()); + + const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0]; + if (!join_expressions_actions_root_node->function) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} join expression expected function", + join_node.formatASTForErrorMessage()); + + size_t left_table_expression_columns_size = left_table_expression_columns.size(); + + Names join_left_actions_names; + join_left_actions_names.reserve(left_table_expression_columns_size); + + NameSet join_left_actions_names_set; + join_left_actions_names_set.reserve(left_table_expression_columns_size); + + for (const auto & left_table_expression_column : left_table_expression_columns) + { + join_left_actions_names.push_back(left_table_expression_column.name); + join_left_actions_names_set.insert(left_table_expression_column.name); + } + + size_t right_table_expression_columns_size = right_table_expression_columns.size(); + + Names join_right_actions_names; + join_right_actions_names.reserve(right_table_expression_columns_size); + + NameSet join_right_actions_names_set; + join_right_actions_names_set.reserve(right_table_expression_columns_size); + + for (const auto & right_table_expression_column : right_table_expression_columns) + { + join_right_actions_names.push_back(right_table_expression_column.name); + join_right_actions_names_set.insert(right_table_expression_column.name); + } + + JoinClausesAndActions result; + result.join_expression_actions = join_expression_actions; + + const auto & function_name = join_expressions_actions_root_node->function->getName(); + if (function_name == "or") + { + for (const auto & child : join_expressions_actions_root_node->children) + { + result.join_clauses.emplace_back(); + + buildJoinClause(join_expression_actions, + child, + join_left_actions_names_set, + join_right_actions_names_set, + join_node, + result.join_clauses.back()); + } + } + else + { + result.join_clauses.emplace_back(); + + buildJoinClause(join_expression_actions, + join_expressions_actions_root_node, + join_left_actions_names_set, + join_right_actions_names_set, + join_node, + result.join_clauses.back()); + } + + auto and_function = FunctionFactory::instance().get("and", planner_context->getQueryContext()); + + auto add_necessary_name_if_needed = [&](JoinTableSide join_table_side, const String & name) + { + auto & necessary_names = join_table_side == JoinTableSide::Left ? join_left_actions_names : join_right_actions_names; + auto & necessary_names_set = join_table_side == JoinTableSide::Left ? join_left_actions_names_set : join_right_actions_names_set; + + auto [_, inserted] = necessary_names_set.emplace(name); + if (inserted) + necessary_names.push_back(name); + }; + + for (auto & join_clause : result.join_clauses) + { + const auto & left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); + if (!left_filter_condition_nodes.empty()) + { + const ActionsDAG::Node * dag_filter_condition_node = nullptr; + + if (left_filter_condition_nodes.size() > 1) + dag_filter_condition_node = &join_expression_actions->addFunction(and_function, left_filter_condition_nodes, {}); + else + dag_filter_condition_node = left_filter_condition_nodes[0]; + + join_clause.getLeftFilterConditionNodes().clear(); + join_clause.addCondition(JoinTableSide::Left, dag_filter_condition_node); + + join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node); + + add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name); + } + + const auto & right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); + if (!right_filter_condition_nodes.empty()) + { + const ActionsDAG::Node * dag_filter_condition_node = nullptr; + + if (right_filter_condition_nodes.size() > 1) + dag_filter_condition_node = &join_expression_actions->addFunction(and_function, right_filter_condition_nodes, {}); + else + dag_filter_condition_node = right_filter_condition_nodes[0]; + + join_clause.getRightFilterConditionNodes().clear(); + join_clause.addCondition(JoinTableSide::Right, dag_filter_condition_node); + + join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node); + + add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name); + } + + assert(join_clause.getLeftKeyNodes().size() == join_clause.getRightKeyNodes().size()); + size_t join_clause_left_key_nodes_size = join_clause.getLeftKeyNodes().size(); + + for (size_t i = 0; i < join_clause_left_key_nodes_size; ++i) + { + auto & left_key_node = join_clause.getLeftKeyNodes()[i]; + auto & right_key_node = join_clause.getRightKeyNodes()[i]; + + if (!left_key_node->result_type->equals(*right_key_node->result_type)) + { + DataTypePtr common_type; + + try + { + common_type = getLeastSupertype(DataTypes{left_key_node->result_type, right_key_node->result_type}); + } + catch (Exception & ex) + { + ex.addMessage("JOIN {} cannot infer common type in ON section for keys. Left key {} type {}. Right key {} type {}", + join_node.formatASTForErrorMessage(), + left_key_node->result_name, + left_key_node->result_type->getName(), + right_key_node->result_name, + right_key_node->result_type->getName()); + } + + ColumnWithTypeAndName cast_column; + cast_column.name = "__constant_" + Field(common_type->getName()).dump(); + cast_column.column = DataTypeString().createColumnConst(0, common_type->getName()); + cast_column.type = std::make_shared(); + + const ActionsDAG::Node * cast_type_constant_node = nullptr; + + if (!left_key_node->result_type->equals(*common_type)) + { + cast_type_constant_node = &join_expression_actions->addColumn(cast_column); + + FunctionCastBase::Diagnostic diagnostic = {left_key_node->result_name, left_key_node->result_name}; + FunctionOverloadResolverPtr func_builder_cast + = CastInternalOverloadResolver::createImpl(diagnostic); + + ActionsDAG::NodeRawConstPtrs children = {left_key_node, cast_type_constant_node}; + left_key_node = &join_expression_actions->addFunction(func_builder_cast, std::move(children), {}); + } + + if (!right_key_node->result_type->equals(*common_type)) + { + if (!cast_type_constant_node) + cast_type_constant_node = &join_expression_actions->addColumn(cast_column); + + FunctionCastBase::Diagnostic diagnostic = {right_key_node->result_name, right_key_node->result_name}; + FunctionOverloadResolverPtr func_builder_cast + = CastInternalOverloadResolver::createImpl(std::move(diagnostic)); + + ActionsDAG::NodeRawConstPtrs children = {right_key_node, cast_type_constant_node}; + right_key_node = &join_expression_actions->addFunction(func_builder_cast, std::move(children), {}); + } + } + + join_expression_actions->addOrReplaceInOutputs(*left_key_node); + join_expression_actions->addOrReplaceInOutputs(*right_key_node); + + add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name); + add_necessary_name_if_needed(JoinTableSide::Right, right_key_node->result_name); + } + } + + result.left_join_expressions_actions = join_expression_actions->clone(); + result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names); + + result.right_join_expressions_actions = join_expression_actions->clone(); + result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); + + return result; +} + +} + +JoinClausesAndActions buildJoinClausesAndActions( + const ColumnsWithTypeAndName & left_stream_columns, + const ColumnsWithTypeAndName & right_stream_columns, + const QueryTreeNodePtr & join_node, + const PlannerContextPtr & planner_context) +{ + auto & join_node_typed = join_node->as(); + if (!join_node_typed.isOnJoinExpression()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} join does not have ON section", + join_node_typed.formatASTForErrorMessage()); + + auto join_expression_input_columns = left_stream_columns; + join_expression_input_columns.insert(join_expression_input_columns.end(), right_stream_columns.begin(), right_stream_columns.end()); + + return buildJoinClausesAndActions(join_expression_input_columns, left_stream_columns, right_stream_columns, join_node_typed, planner_context); +} + +} diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h new file mode 100644 index 00000000000..121a2ea6ce3 --- /dev/null +++ b/src/Planner/PlannerJoins.h @@ -0,0 +1,149 @@ +#pragma once + +#include +#include + +#include + +#include + +namespace DB +{ + +/** Join clause represent single JOIN ON section clause. + * Join clause consits of JOIN keys and conditions. + * + * JOIN can contain multiple clauses in JOIN ON section. + * Example: SELECT * FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id OR t1.value = t2.value; + * t1.id = t2.id is first clause. + * t1.value = t2.value is second clause. + * + * JOIN ON section can also contain condition inside clause. + * Example: SELECT * FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id AND t1.id > 0 AND t2.id > 0; + * t1.id = t2.id AND t1.id > 0 AND t2.id > 0 is first clause. + * t1.id = t2.id is JOIN keys section. + * t1.id > 0 is left table condition. + * t2.id > 0 is right table condition. + * + * Additionally not only conditions, but JOIN keys can be represented as expressions. + * Example: SELECT * FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON toString(t1.id) = toString(t2.id). + * toString(t1.id) = toString(t2.id) is JOIN keys section. Where toString(t1.id) is left key, and toString(t2.id) is right key. + * + * During query planning JOIN ON section must be represented using join clause structure. It is important to split + * keys and conditions. And for each action detect from which stream it can be performed. + * + * We have 2 streams, left stream and right stream. + * We split JOIN ON section expressions actions in two parts left join expression actions and right join expression actions. + * Left join expresion actions must be used to calculate necessary actions for left stream. + * Right join expression actions must be used to calculate necessary actions for right stream. + */ +class PlannerContext; +using PlannerContextPtr = std::shared_ptr; + +/// Single JOIN ON section clause representation +class JoinClause +{ +public: + /// Add keys + void addKey(const ActionsDAG::Node * left_key_node, const ActionsDAG::Node * right_key_node) + { + left_key_nodes.emplace_back(left_key_node); + right_key_nodes.emplace_back(right_key_node); + } + + /// Add condition for table side + void addCondition(JoinTableSide table_side, const ActionsDAG::Node * condition_node) + { + auto & filter_condition_nodes = table_side == JoinTableSide::Left ? left_filter_condition_nodes : right_filter_condition_nodes; + filter_condition_nodes.push_back(condition_node); + } + + /// Get left key nodes + const ActionsDAG::NodeRawConstPtrs & getLeftKeyNodes() const + { + return left_key_nodes; + } + + /// Get right key nodes + const ActionsDAG::NodeRawConstPtrs & getRightKeyNodes() const + { + return right_key_nodes; + } + + /// Get left key nodes + ActionsDAG::NodeRawConstPtrs & getLeftKeyNodes() + { + return left_key_nodes; + } + + /// Get right key nodes + ActionsDAG::NodeRawConstPtrs & getRightKeyNodes() + { + return right_key_nodes; + } + + /// Get left filter condition nodes + const ActionsDAG::NodeRawConstPtrs & getLeftFilterConditionNodes() const + { + return left_filter_condition_nodes; + } + + /// Get right filter condition nodes + const ActionsDAG::NodeRawConstPtrs & getRightFilterConditionNodes() const + { + return right_filter_condition_nodes; + } + + ActionsDAG::NodeRawConstPtrs & getLeftFilterConditionNodes() + { + return left_filter_condition_nodes; + } + + /// Get right filter condition nodes + ActionsDAG::NodeRawConstPtrs & getRightFilterConditionNodes() + { + return right_filter_condition_nodes; + } + + /// Dump clause into buffer + void dump(WriteBuffer & buffer) const; + + /// Dump clause + String dump() const; + +private: + ActionsDAG::NodeRawConstPtrs left_key_nodes; + ActionsDAG::NodeRawConstPtrs right_key_nodes; + + ActionsDAG::NodeRawConstPtrs left_filter_condition_nodes; + ActionsDAG::NodeRawConstPtrs right_filter_condition_nodes; +}; + +using JoinClauses = std::vector; + +struct JoinClausesAndActions +{ + /// Join clauses. Actions dag nodes point into join_expression_actions. + JoinClauses join_clauses; + /// Whole JOIN ON section expressions + ActionsDAGPtr join_expression_actions; + /// Left join expressions actions + ActionsDAGPtr left_join_expressions_actions; + /// Right join expressions actions + ActionsDAGPtr right_join_expressions_actions; +}; + +/** Calculate join clauses and actions for JOIN ON section. + * + * left_table_expression_columns - columns from left join stream. + * right_table_expression_columns - columns from right join stream. + * join_node - join query tree node. + * planner_context - planner context. + */ +JoinClausesAndActions buildJoinClausesAndActions( + const ColumnsWithTypeAndName & left_stream_columns, + const ColumnsWithTypeAndName & right_stream_columns, + const QueryTreeNodePtr & join_node, + const PlannerContextPtr & planner_context); + +} diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp new file mode 100644 index 00000000000..fdd976f1374 --- /dev/null +++ b/src/Planner/Utils.cpp @@ -0,0 +1,59 @@ +#include + +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; +} + +String dumpQueryPlan(QueryPlan & query_plan) +{ + WriteBufferFromOwnString query_plan_buffer; + query_plan.explainPlan(query_plan_buffer, QueryPlan::ExplainPlanOptions{true, true, true, true}); + return query_plan_buffer.str(); +} + +String dumpQueryPipeline(QueryPlan & query_plan) +{ + QueryPlan::ExplainPipelineOptions explain_pipeline; + WriteBufferFromOwnString query_pipeline_buffer; + query_plan.explainPipeline(query_pipeline_buffer, explain_pipeline); + return query_pipeline_buffer.str(); +} + +Block buildCommonHeaderForUnion(const Blocks & queries_headers) +{ + size_t num_selects = queries_headers.size(); + Block common_header = queries_headers.front(); + size_t num_columns = common_header.columns(); + + for (size_t query_num = 1; query_num < num_selects; ++query_num) + { + if (queries_headers.at(query_num).columns() != num_columns) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Different number of columns in UNION elements: {} and {}", + common_header.dumpNames(), + queries_headers[query_num].dumpNames()); + } + + std::vector columns(num_selects); + + for (size_t column_num = 0; column_num < num_columns; ++column_num) + { + for (size_t i = 0; i < num_selects; ++i) + columns[i] = &queries_headers[i].getByPosition(column_num); + + ColumnWithTypeAndName & result_elem = common_header.getByPosition(column_num); + result_elem = getLeastSuperColumn(columns); + } + + return common_header; +} + +} diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h new file mode 100644 index 00000000000..2e294e0e156 --- /dev/null +++ b/src/Planner/Utils.h @@ -0,0 +1,18 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Dump query plan +String dumpQueryPlan(QueryPlan & query_plan); + +/// Dump query plan result pipeline +String dumpQueryPipeline(QueryPlan & query_plan); + +/// Build common header for UNION query +Block buildCommonHeaderForUnion(const Blocks & queries_headers); + +} diff --git a/src/Planner/examples/CMakeLists.txt b/src/Planner/examples/CMakeLists.txt new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Planner/tests/CMakeLists.txt b/src/Planner/tests/CMakeLists.txt new file mode 100644 index 00000000000..e69de29bb2d From 2995fb7944486c6e68a913413c831f282d75e6ca Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 24 Aug 2022 16:50:53 +0200 Subject: [PATCH 073/188] Added support for building sets for IN function --- src/Analyzer/QueryAnalysisPass.cpp | 44 +- src/Analyzer/QueryAnalysisPass.h | 10 +- .../InterpreterSelectQueryAnalyzer.cpp | 6 +- src/Planner/ActionsChain.cpp | 5 +- src/Planner/Planner.cpp | 79 +++- src/Planner/Planner.h | 2 +- src/Planner/PlannerActionsVisitor.cpp | 375 ++++++++++++------ src/Planner/PlannerActionsVisitor.h | 37 +- src/Planner/PlannerContext.h | 43 +- src/Planner/PlannerJoins.cpp | 4 +- src/Planner/Utils.cpp | 19 + src/Planner/Utils.h | 8 + ...76_function_in_function_subquery.reference | 40 ++ .../02376_function_in_function_subquery.sql | 52 +++ 14 files changed, 546 insertions(+), 178 deletions(-) create mode 100644 tests/queries/0_stateless/02376_function_in_function_subquery.reference create mode 100644 tests/queries/0_stateless/02376_function_in_function_subquery.sql diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 46fa43b3be7..fc6eeef5771 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -2276,7 +2276,7 @@ void QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const Qu * 1. Resolve function parameters. Validate that each function parameter must be constant node. * 2. Resolve function arguments list, lambda expressions are allowed as function arguments. * 3. Initialize argument_columns, argument_types, function_lambda_arguments_indexes arrays from function arguments. - * 4. Try to resolve function name as identifier as function. + * 4. Try to resolve function node name identifier as function. * 5. If function name identifier was not resolved as function, try to lookup lambda from sql user defined functions factory. * 6. If function was resolve as lambda from step 4, or 5, then resolve lambda using function arguments and replace function node with lambda result. * After than function node is resolved. @@ -2332,6 +2332,43 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc resolveExpressionNodeList(function_node.getArgumentsNode(), scope, true /*allow_lambda_expression*/, is_special_function_in /*allow_table_expression*/); + /// Replace right IN function argument if it is table or table function with subquery that read ordinary columns + if (is_special_function_in) + { + auto & in_second_argument = function_node.getArguments().getNodes().at(1); + auto * table_node = in_second_argument->as(); + auto * table_function_node = in_second_argument->as(); + auto * query_node = in_second_argument->as(); + auto * union_node = in_second_argument->as(); + + if (table_node || table_function_node) + { + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); + auto column_nodes_to_select = std::make_shared(); + + for (auto & column : columns_to_select) + column_nodes_to_select->getNodes().push_back(std::make_shared(column, in_second_argument)); + + auto in_second_argument_query_node = std::make_shared(); + in_second_argument_query_node->setIsSubquery(true); + in_second_argument_query_node->getProjectionNode() = std::move(column_nodes_to_select); + in_second_argument_query_node->getJoinTree() = std::move(in_second_argument); + + in_second_argument = std::move(in_second_argument_query_node); + } + else if (query_node) + { + IdentifierResolveScope subquery_scope(in_second_argument, &scope /*parent_scope*/); + resolveQuery(in_second_argument, subquery_scope); + } + else if (union_node) + { + IdentifierResolveScope subquery_scope(in_second_argument, &scope /*parent_scope*/); + resolveUnion(in_second_argument, subquery_scope); + } + } + /// Initialize function argument columns ColumnsWithTypeAndName argument_columns; @@ -2358,8 +2395,9 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc function_lambda_arguments_indexes.push_back(function_argument_index); } else if (is_special_function_in && (function_argument->getNodeType() == QueryTreeNodeType::TABLE || - function_argument->getNodeType() == QueryTreeNodeType::TABLE_FUNCTION || - function_argument->getNodeType() == QueryTreeNodeType::QUERY)) + function_argument->getNodeType() == QueryTreeNodeType::TABLE_FUNCTION || + function_argument->getNodeType() == QueryTreeNodeType::QUERY || + function_argument->getNodeType() == QueryTreeNodeType::UNION)) { argument_column.type = std::make_shared(); } diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index bf56b739a90..7da4d9f2971 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -29,10 +29,16 @@ namespace DB * 8. Special functions handling: * Function `untuple` is handled properly. * Function `arrayJoin` is handled properly. + * * For function `grouping` arguments are resolved, but it is planner responsibility to initialize it with concrete grouping function * based on group by kind and group by keys positions. - * For function `in` and its variations arguments are resolved, but sets are not build. If left and right arguments are constants - * constant folding is performed. + * + * For function `in` and its variations arguments are resolved, but sets are not build. + * If left and right arguments are constants constant folding is performed. + * If right argument resolved as table function, or table, it is replaced with query that read only ordinary columns from underlying + * storage. + * Example: SELECT id FROM test_table WHERE id IN test_table_other; + * Result: SELECT id FROM test_table WHERE id IN (SELECT test_table_column FROM test_table_other); */ class QueryAnalysisPass final : public IQueryTreePass { diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index b236af15df8..94f1739fe3c 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -117,19 +117,19 @@ InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( Block InterpreterSelectQueryAnalyzer::getSampleBlock() { - planner.initializeQueryPlanIfNeeded(); + planner.buildQueryPlanIfNeeded(); return planner.getQueryPlan().getCurrentDataStream().header; } QueryPlan && InterpreterSelectQueryAnalyzer::extractQueryPlan() && { - planner.initializeQueryPlanIfNeeded(); + planner.buildQueryPlanIfNeeded(); return std::move(planner).extractQueryPlan(); } BlockIO InterpreterSelectQueryAnalyzer::execute() { - planner.initializeQueryPlanIfNeeded(); + planner.buildQueryPlanIfNeeded(); auto & query_plan = planner.getQueryPlan(); QueryPlanOptimizationSettings optimization_settings; diff --git a/src/Planner/ActionsChain.cpp b/src/Planner/ActionsChain.cpp index 97f2119b2d7..c7aa8361ed4 100644 --- a/src/Planner/ActionsChain.cpp +++ b/src/Planner/ActionsChain.cpp @@ -20,9 +20,8 @@ void ActionsChainStep::finalizeInputAndOutputColumns(const NameSet & child_input for (const auto & node : actions->getNodes()) { - auto it = child_input_columns.find(node.result_name); - - if (it == child_input_columns.end()) + auto it = child_input_columns_copy.find(node.result_name); + if (it == child_input_columns_copy.end()) continue; child_required_output_columns_names.insert(node.result_name); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 820af998823..0ff2ad75442 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include @@ -75,7 +76,6 @@ namespace ErrorCodes * TODO: Support ORDER BY, LIMIT * TODO: Support WINDOW FUNCTIONS * TODO: Support DISTINCT - * TODO: Support building sets for IN functions * TODO: Support trivial count optimization * TODO: Support totals, extremes * TODO: Support projections @@ -132,7 +132,8 @@ public: default: { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected query, table, table function, join or array join query node. Actual {}", + "Expected query, union, table, table function, join or array join query node. Actual {} {}", + join_tree_node->getNodeTypeName(), join_tree_node->formatASTForErrorMessage()); } } @@ -214,8 +215,8 @@ using CollectSourceColumnsVisitor = CollectSourceColumnsMatcher::Visitor; ActionsDAGPtr convertExpressionNodeIntoDAG(const QueryTreeNodePtr & expression_node, const ColumnsWithTypeAndName & inputs, const PlannerContextPtr & planner_context) { ActionsDAGPtr action_dag = std::make_shared(inputs); - PlannerActionsVisitor actions_visitor(action_dag, planner_context); - auto expression_dag_index_nodes = actions_visitor.visit(expression_node); + PlannerActionsVisitor actions_visitor(planner_context); + auto expression_dag_index_nodes = actions_visitor.visit(action_dag, expression_node); action_dag->getOutputs().clear(); for (auto & expression_dag_index_node : expression_dag_index_nodes) @@ -294,8 +295,9 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, } else if (query_node || union_node) { - Planner subquery_planner(table_expression, select_query_options, planner_context->getQueryContext(), planner_context->getGlobalPlannerContext()); - subquery_planner.initializeQueryPlanIfNeeded(); + auto subquery_options = select_query_options.subquery(); + Planner subquery_planner(table_expression, subquery_options, planner_context->getQueryContext(), planner_context->getGlobalPlannerContext()); + subquery_planner.buildQueryPlanIfNeeded(); query_plan = std::move(subquery_planner).extractQueryPlan(); } else @@ -604,7 +606,7 @@ QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression, auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); ActionsDAGPtr array_join_action_dag = std::make_shared(left_plan_output_columns); - PlannerActionsVisitor actions_visitor(array_join_action_dag, planner_context); + PlannerActionsVisitor actions_visitor(planner_context); NameSet array_join_columns; for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) @@ -613,7 +615,7 @@ QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression, const auto & array_join_column_name = array_join_expression_column.getColumnName(); array_join_columns.insert(array_join_column_name); - auto expression_dag_index_nodes = actions_visitor.visit(array_join_expression_column.getExpressionOrThrow()); + auto expression_dag_index_nodes = actions_visitor.visit(array_join_action_dag, array_join_expression_column.getExpressionOrThrow()); for (auto & expression_dag_index_node : expression_dag_index_nodes) { const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_name); @@ -670,6 +672,51 @@ QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, } } +void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, const SelectQueryOptions & select_query_options, const PlannerContextPtr & planner_context) +{ + if (select_query_options.is_subquery) + return; + + SubqueriesForSets subqueries_for_sets; + const auto & subquery_node_to_sets = planner_context->getGlobalPlannerContext()->getSubqueryNodesForSets(); + + for (auto [key, subquery_node_for_set] : subquery_node_to_sets) + { + /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. + * Because the result of this query is not the result of the entire query. + * Constraints work instead + * max_rows_in_set, max_bytes_in_set, set_overflow_mode, + * max_rows_in_join, max_bytes_in_join, join_overflow_mode, + * which are checked separately (in the Set, Join objects). + */ + auto subquery_context = Context::createCopy(planner_context->getQueryContext()); + Settings subquery_settings = planner_context->getQueryContext()->getSettings(); + subquery_settings.max_result_rows = 0; + subquery_settings.max_result_bytes = 0; + /// The calculation of `extremes` does not make sense and is not necessary (if you do it, then the `extremes` of the subquery can be taken instead of the whole query). + subquery_settings.extremes = false; + subquery_context->setSettings(subquery_settings); + + auto subquery_options = select_query_options.subquery(); + Planner subquery_planner( + subquery_node_for_set.subquery_node, + subquery_options, + planner_context->getQueryContext(), + planner_context->getGlobalPlannerContext()); + subquery_planner.buildQueryPlanIfNeeded(); + + SubqueryForSet subquery_for_set; + subquery_for_set.set = subquery_node_for_set.set; + subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + + subqueries_for_sets.emplace(key, std::move(subquery_for_set)); + } + + const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); + SizeLimits limits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode); + addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), limits, planner_context->getQueryContext()); +} + } Planner::Planner(const QueryTreeNodePtr & query_tree_, @@ -704,7 +751,7 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_, query_tree->formatASTForErrorMessage()); } -void Planner::initializeQueryPlanIfNeeded() +void Planner::buildQueryPlanIfNeeded() { if (query_plan.isInitialized()) return; @@ -728,7 +775,7 @@ void Planner::initializeQueryPlanIfNeeded() for (auto & query_node : union_query_tree->getQueries().getNodes()) { Planner query_planner(query_node, select_query_options, current_context); - query_planner.initializeQueryPlanIfNeeded(); + query_planner.buildQueryPlanIfNeeded(); auto query_node_plan = std::make_unique(std::move(query_planner).extractQueryPlan()); query_plans_headers.push_back(query_node_plan->getCurrentDataStream().header); query_plans.push_back(std::move(query_node_plan)); @@ -741,7 +788,10 @@ void Planner::initializeQueryPlanIfNeeded() for (auto & query_node_plan : query_plans) { if (blocksHaveEqualStructure(query_node_plan->getCurrentDataStream().header, union_common_header)) + { + query_plans_streams.push_back(query_node_plan->getCurrentDataStream()); continue; + } auto actions_dag = ActionsDAG::makeConvertingActions( query_node_plan->getCurrentDataStream().header.getColumnsWithTypeAndName(), @@ -792,11 +842,10 @@ void Planner::initializeQueryPlanIfNeeded() } auto & query_node = query_tree->as(); - auto query = query_node.toAST(); SelectQueryInfo select_query_info; - select_query_info.original_query = query; - select_query_info.query = query; + select_query_info.original_query = queryNodeToSelectQuery(query_tree); + select_query_info.query = select_query_info.original_query; CollectTableExpressionIdentifiersVisitor collect_table_expression_identifiers_visitor; collect_table_expression_identifiers_visitor.visit(query_node.getJoinTree(), *planner_context); @@ -859,7 +908,7 @@ void Planner::initializeQueryPlanIfNeeded() size_t projection_action_step_index = actions_chain.getLastStepIndex(); // std::cout << "Chain dump before finalize" << std::endl; - // std::cout << planner_context.actions_chain.dump() << std::endl; + // std::cout << actions_chain.dump() << std::endl; actions_chain.finalize(); @@ -884,6 +933,8 @@ void Planner::initializeQueryPlanIfNeeded() auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), actions_chain[projection_action_step_index]->getActions()); projection_step->setStepDescription("Projection"); query_plan.addStep(std::move(projection_step)); + + addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context); } } diff --git a/src/Planner/Planner.h b/src/Planner/Planner.h index 19f5f582fcf..e110bc044f9 100644 --- a/src/Planner/Planner.h +++ b/src/Planner/Planner.h @@ -40,7 +40,7 @@ public: return query_plan; } - void initializeQueryPlanIfNeeded(); + void buildQueryPlanIfNeeded(); QueryPlan && extractQueryPlan() && { diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index cdeb8ef0225..79241091f03 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -1,15 +1,24 @@ #include #include +#include #include #include #include #include -#include + +#include + +#include +#include #include #include + #include +#include + +#include namespace DB { @@ -22,7 +31,10 @@ namespace ErrorCodes extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } -class PlannerActionsVisitor::ActionsScopeNode +namespace +{ + +class ActionsScopeNode { public: explicit ActionsScopeNode(ActionsDAGPtr actions_dag_, QueryTreeNodePtr scope_node_) @@ -130,15 +142,42 @@ private: QueryTreeNodePtr scope_node; }; -PlannerActionsVisitor::PlannerActionsVisitor(ActionsDAGPtr actions_dag, const PlannerContextPtr & planner_context_) +class PlannerActionsVisitorImpl +{ +public: + PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag, const PlannerContextPtr & planner_context_); + + ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node); + +private: + using NodeNameAndNodeMinLevel = std::pair; + + NodeNameAndNodeMinLevel visitImpl(QueryTreeNodePtr node); + + NodeNameAndNodeMinLevel visitColumn(const QueryTreeNodePtr & node); + + NodeNameAndNodeMinLevel visitConstant(const QueryTreeNodePtr & node); + + NodeNameAndNodeMinLevel visitLambda(const QueryTreeNodePtr & node); + + NodeNameAndNodeMinLevel makeSetForInFunction(const QueryTreeNodePtr & node); + + NodeNameAndNodeMinLevel visitFunction(const QueryTreeNodePtr & node); + + String calculateActionsDAGNodeName(const IQueryTreeNode * node); + + std::vector actions_stack; + std::unordered_map node_to_node_name; + const PlannerContextPtr planner_context; +}; + +PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag, const PlannerContextPtr & planner_context_) : planner_context(planner_context_) { - actions_stack.emplace_back(std::make_unique(std::move(actions_dag), nullptr)); + actions_stack.emplace_back(std::move(actions_dag), nullptr); } -PlannerActionsVisitor::~PlannerActionsVisitor() = default; - -ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(QueryTreeNodePtr expression_node) +ActionsDAG::NodeRawConstPtrs PlannerActionsVisitorImpl::visit(QueryTreeNodePtr expression_node) { ActionsDAG::NodeRawConstPtrs result; @@ -147,19 +186,19 @@ ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(QueryTreeNodePtr expre for (auto & node : expression_list_node->getNodes()) { auto [node_name, _] = visitImpl(node); - result.push_back(actions_stack.front()->getNodeOrThrow(node_name)); + result.push_back(actions_stack.front().getNodeOrThrow(node_name)); } } else { auto [node_name, _] = visitImpl(expression_node); - result.push_back(actions_stack.front()->getNodeOrThrow(node_name)); + result.push_back(actions_stack.front().getNodeOrThrow(node_name)); } return result; } -PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitImpl(QueryTreeNodePtr node) +PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitImpl(QueryTreeNodePtr node) { if (auto * column_node = node->as()) return visitColumn(node); @@ -173,18 +212,18 @@ PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitImpl( node->formatASTForErrorMessage()); } -PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitColumn(const QueryTreeNodePtr & node) +PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitColumn(const QueryTreeNodePtr & node) { - auto column_node_name = getActionsDAGNodeName(node.get()); + auto column_node_name = calculateActionsDAGNodeName(node.get()); const auto & column_node = node->as(); Int64 actions_stack_size = static_cast(actions_stack.size() - 1); for (Int64 i = actions_stack_size; i >= 0; --i) { - actions_stack[i]->addInputColumnIfNecessary(column_node_name, column_node.getColumnType()); + actions_stack[i].addInputColumnIfNecessary(column_node_name, column_node.getColumnType()); if (column_node.getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA - && actions_stack[i]->getScopeNode().get() == column_node.getColumnSource().get()) + && actions_stack[i].getScopeNode().get() == column_node.getColumnSource().get()) { return {column_node_name, i}; } @@ -193,9 +232,9 @@ PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitColum return {column_node_name, 0}; } -PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitConstant(const QueryTreeNodePtr & node) +PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitConstant(const QueryTreeNodePtr & node) { - auto constant_node_name = getActionsDAGNodeName(node.get()); + auto constant_node_name = calculateActionsDAGNodeName(node.get()); const auto & constant_node = node->as(); const auto & literal = constant_node.getConstantValue(); @@ -204,19 +243,19 @@ PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitConst column.type = constant_node.getResultType(); column.column = column.type->createColumnConst(1, literal); - actions_stack[0]->addConstantIfNecessary(constant_node_name, column); + actions_stack[0].addConstantIfNecessary(constant_node_name, column); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node->addInputConstantColumnIfNecessary(constant_node_name, column); + actions_stack_node.addInputConstantColumnIfNecessary(constant_node_name, column); } return {constant_node_name, 0}; } -PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitLambda(const QueryTreeNodePtr & node) +PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitLambda(const QueryTreeNodePtr & node) { auto & lambda_node = node->as(); auto result_type = lambda_node.getResultType(); @@ -237,10 +276,10 @@ PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitLambd size_t previous_scope_node_actions_stack_index = actions_stack.size() - 1; auto lambda_actions_dag = std::make_shared(); - actions_stack.emplace_back(std::make_unique(lambda_actions_dag, node)); + actions_stack.emplace_back(lambda_actions_dag, node); auto [lambda_expression_node_name, level] = visitImpl(lambda_node.getExpression()); - lambda_actions_dag->getOutputs().push_back(actions_stack.back()->getNodeOrThrow(lambda_expression_node_name)); + lambda_actions_dag->getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name)); lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name)); auto expression_actions_settings = ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::yes); @@ -259,12 +298,12 @@ PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitLambd if (it == lambda_argument_names.end()) { - lambda_children.push_back(actions_stack[previous_scope_node_actions_stack_index]->getNodeOrThrow(required_column_name)); + lambda_children.push_back(actions_stack[previous_scope_node_actions_stack_index].getNodeOrThrow(required_column_name)); captured_column_names.push_back(required_column_name); } } - auto lambda_node_name = getActionsDAGNodeName(node.get()); + auto lambda_node_name = calculateActionsDAGNodeName(node.get()); auto function_capture = std::make_shared( lambda_actions, captured_column_names, lambda_arguments_names_and_types, result_type, lambda_expression_node_name); actions_stack.pop_back(); @@ -272,22 +311,93 @@ PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitLambd if (level == actions_stack.size()) --level; - actions_stack[level]->addFunctionIfNecessary(lambda_node_name, lambda_children, function_capture); + actions_stack[level].addFunctionIfNecessary(lambda_node_name, lambda_children, function_capture); size_t actions_stack_size = actions_stack.size(); for (size_t i = level + 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node->addInputColumnIfNecessary(lambda_node_name, result_type); + actions_stack_node.addInputColumnIfNecessary(lambda_node_name, result_type); } return {lambda_node_name, level}; } -PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitFunction(const QueryTreeNodePtr & node) +PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::makeSetForInFunction(const QueryTreeNodePtr & node) { - auto function_node_name = getActionsDAGNodeName(node.get()); const auto & function_node = node->as(); + auto in_first_argument = function_node.getArguments().getNodes().at(0); + auto in_second_argument = function_node.getArguments().getNodes().at(1); + auto in_second_argument_node_type = in_second_argument->getNodeType(); + + auto set_source_hash = in_second_argument->getTreeHash(); + String set_key = "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); + auto prepared_set = planner_context->getGlobalPlannerContext()->getSet(set_key); + + if (!prepared_set) + { + if (in_second_argument_node_type == QueryTreeNodeType::QUERY || + in_second_argument_node_type == QueryTreeNodeType::UNION) + { + const auto & settings = planner_context->getQueryContext()->getSettingsRef(); + SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; + bool tranform_null_in = settings.transform_null_in; + + auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); + + planner_context->getGlobalPlannerContext()->registerSet(set_key, set); + planner_context->getGlobalPlannerContext()->registerSubqueryNodeForSet(set_key, SubqueryNodeForSet{in_second_argument, set}); + + prepared_set = std::move(set); + } + else if (in_second_argument_node_type == QueryTreeNodeType::CONSTANT) + { + auto & in_second_argument_constant_node = in_second_argument->as(); + + const auto & settings = planner_context->getQueryContext()->getSettingsRef(); + + auto set = makeSetForConstantValue( + in_first_argument->getResultType(), + in_second_argument_constant_node.getResultType(), + in_second_argument_constant_node.getConstantValue(), + settings); + + planner_context->getGlobalPlannerContext()->registerSet(set_key, set); + prepared_set = std::move(set); + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Function IN is supported only if second argument is constant or table expression"); + } + } + + auto column_set = ColumnSet::create(1, std::move(prepared_set)); + auto column_set_const = ColumnConst::create(std::move(column_set), 1); + + ColumnWithTypeAndName column; + column.name = set_key; + column.type = std::make_shared(); + column.column = std::move(column_set_const); + + actions_stack[0].addConstantIfNecessary(set_key, column); + + size_t actions_stack_size = actions_stack.size(); + for (size_t i = 1; i < actions_stack_size; ++i) + { + auto & actions_stack_node = actions_stack[i]; + actions_stack_node.addInputConstantColumnIfNecessary(set_key, column); + } + + node_to_node_name.emplace(in_second_argument.get(), set_key); + + return {set_key, 0}; +} + +PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitFunction(const QueryTreeNodePtr & node) +{ + const auto & function_node = node->as(); + std::optional in_function_second_argument_node_name_with_level; if (function_node.getFunctionName() == "grouping") { @@ -305,20 +415,7 @@ PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitFunct } else if (isNameOfInFunction(function_node.getFunctionName())) { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function IN is not supported"); - } - - if (function_node.isAggregateFunction()) - { - size_t actions_stack_size = actions_stack.size(); - - for (size_t i = 0; i < actions_stack_size; ++i) - { - auto & actions_stack_node = actions_stack[i]; - actions_stack_node->addInputColumnIfNecessary(function_node_name, function_node.getResultType()); - } - - return {function_node_name, 0}; + in_function_second_argument_node_name_with_level = makeSetForInFunction(node); } const auto & function_arguments = function_node.getArguments().getNodes(); @@ -328,8 +425,18 @@ PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitFunct function_arguments_node_names.reserve(function_arguments_size); size_t level = 0; - for (const auto & argument : function_arguments) + for (size_t function_argument_index = 0; function_argument_index < function_arguments_size; ++function_argument_index) { + if (in_function_second_argument_node_name_with_level && function_argument_index == 1) + { + auto & [node_name, node_min_level] = *in_function_second_argument_node_name_with_level; + function_arguments_node_names.push_back(std::move(node_name)); + level = std::max(level, node_min_level); + continue; + } + + const auto & argument = function_arguments[function_argument_index]; + if (argument->getNodeType() == QueryTreeNodeType::LAMBDA) { auto [node_name, node_min_level] = visitLambda(argument); @@ -343,124 +450,158 @@ PlannerActionsVisitor::NodeNameAndNodeMinLevel PlannerActionsVisitor::visitFunct level = std::max(level, node_min_level); } + auto function_node_name = calculateActionsDAGNodeName(node.get()); + + if (function_node.isAggregateFunction()) + { + size_t actions_stack_size = actions_stack.size(); + + for (size_t i = 0; i < actions_stack_size; ++i) + { + auto & actions_stack_node = actions_stack[i]; + actions_stack_node.addInputColumnIfNecessary(function_node_name, function_node.getResultType()); + } + + return {function_node_name, 0}; + } + ActionsDAG::NodeRawConstPtrs children; children.reserve(function_arguments_size); for (auto & function_argument_node_name : function_arguments_node_names) - children.push_back(actions_stack[level]->getNodeOrThrow(function_argument_node_name)); + children.push_back(actions_stack[level].getNodeOrThrow(function_argument_node_name)); if (function_node.getFunctionName() == "arrayJoin") { if (level != 0) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression in arrayJoin cannot depend on lambda argument: {} ", function_arguments_node_names.at(0)); - actions_stack[level]->addArrayJoinIfNecessary(function_node_name, children.at(0)); + actions_stack[level].addArrayJoinIfNecessary(function_node_name, children.at(0)); } else { - actions_stack[level]->addFunctionIfNecessary(function_node_name, children, function_node.getFunction()); + actions_stack[level].addFunctionIfNecessary(function_node_name, children, function_node.getFunction()); } size_t actions_stack_size = actions_stack.size(); for (size_t i = level + 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node->addInputColumnIfNecessary(function_node_name, function_node.getResultType()); + actions_stack_node.addInputColumnIfNecessary(function_node_name, function_node.getResultType()); } return {function_node_name, level}; } - String PlannerActionsVisitor::getActionsDAGNodeName(const IQueryTreeNode * node) const +String PlannerActionsVisitorImpl::calculateActionsDAGNodeName(const IQueryTreeNode * node) +{ + auto it = node_to_node_name.find(node); + + if (it != node_to_node_name.end()) + return it->second; + + String result; + auto node_type = node->getNodeType(); + + switch (node_type) { - String result; - auto node_type = node->getNodeType(); - - switch (node_type) + case QueryTreeNodeType::COLUMN: { - case QueryTreeNodeType::COLUMN: + const auto * column_identifier = planner_context->getColumnNodeIdentifierOrNull(node); + result = column_identifier ? *column_identifier : node->getName(); + + break; + } + case QueryTreeNodeType::CONSTANT: + { + std::string constant_value_dump = node->as().getConstantValue().dump(); + result = "__constant_" + constant_value_dump; + break; + } + case QueryTreeNodeType::FUNCTION: + { + const auto & function_node = node->as(); + + WriteBufferFromOwnString buffer; + buffer << "__function_" + function_node.getFunctionName(); + + const auto & function_parameters_nodes = function_node.getParameters().getNodes(); + + if (!function_parameters_nodes.empty()) { - const auto * column_identifier = planner_context->getColumnNodeIdentifierOrNull(node); - result = column_identifier ? *column_identifier : node->getName(); - - break; - } - case QueryTreeNodeType::CONSTANT: - { - std::string constant_value_dump = node->as().getConstantValue().dump(); - result = "__constant_" + constant_value_dump; - break; - } - case QueryTreeNodeType::FUNCTION: - { - const auto & function_node = node->as(); - - WriteBufferFromOwnString buffer; - buffer << "__function_" + function_node.getFunctionName(); - - const auto & function_parameters_nodes = function_node.getParameters().getNodes(); - - if (!function_parameters_nodes.empty()) - { - buffer << '('; - - size_t function_parameters_nodes_size = function_parameters_nodes.size(); - for (size_t i = 0; i < function_parameters_nodes_size; ++i) - { - const auto & function_parameter_node = function_parameters_nodes[i]; - getActionsDAGNodeName(function_parameter_node.get()); - - if (i + 1 != function_parameters_nodes_size) - buffer << ", "; - } - - buffer << ')'; - } - - const auto & function_arguments_nodes = function_node.getArguments().getNodes(); - buffer << '('; - size_t function_arguments_nodes_size = function_arguments_nodes.size(); - for (size_t i = 0; i < function_arguments_nodes_size; ++i) + size_t function_parameters_nodes_size = function_parameters_nodes.size(); + for (size_t i = 0; i < function_parameters_nodes_size; ++i) { - const auto & function_argument_node = function_arguments_nodes[i]; - buffer << getActionsDAGNodeName(function_argument_node.get()); + const auto & function_parameter_node = function_parameters_nodes[i]; + calculateActionsDAGNodeName(function_parameter_node.get()); - if (i + 1 != function_arguments_nodes_size) + if (i + 1 != function_parameters_nodes_size) buffer << ", "; } buffer << ')'; + } - result = buffer.str(); - break; - } - case QueryTreeNodeType::QUERY: - { - auto query_hash = node->getTreeHash(); + const auto & function_arguments_nodes = function_node.getArguments().getNodes(); - result = "__subquery_" + std::to_string(query_hash.first) + '_' + std::to_string(query_hash.second); - break; - } - case QueryTreeNodeType::LAMBDA: - { - auto lambda_hash = node->getTreeHash(); + buffer << '('; - result = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); - break; - } - default: + size_t function_arguments_nodes_size = function_arguments_nodes.size(); + for (size_t i = 0; i < function_arguments_nodes_size; ++i) { - result = node->getName(); - break; + const auto & function_argument_node = function_arguments_nodes[i]; + buffer << calculateActionsDAGNodeName(function_argument_node.get()); + + if (i + 1 != function_arguments_nodes_size) + buffer << ", "; } + + buffer << ')'; + + result = buffer.str(); + break; } + case QueryTreeNodeType::QUERY: + { + auto query_hash = node->getTreeHash(); - return result; + result = "__subquery_" + std::to_string(query_hash.first) + '_' + std::to_string(query_hash.second); + break; + } + case QueryTreeNodeType::LAMBDA: + { + auto lambda_hash = node->getTreeHash(); + + result = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); + break; + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Actions visitor invalid query tree node {}", + node->formatASTForErrorMessage()); + } } + node_to_node_name.emplace(node, result); + + return result; +} + +} + +PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_context_) + : planner_context(planner_context_) +{} + +ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAGPtr actions_dag, QueryTreeNodePtr expression_node) +{ + PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context); + return actions_visitor_impl.visit(expression_node); +} + } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 571d8fe8be3..13cdca36d5b 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -18,32 +18,29 @@ namespace DB class PlannerContext; using PlannerContextPtr = std::shared_ptr; +/** Planner actions visitor is responsible for adding necessary actions to calculate query tree expression node + * into actions dag. + * + * Column name to identifier map in planner context must be already initialized. + * Identifiers in this map are used as action dag node names for column query tree nodes. + * + * During actions build, there is special handling for following functions: + * 1. Aggregate functions are added in actions dag as INPUT nodes. Aggregate functions arguments are not added. + * 2. For function `in` and its variants, planner context is populated with necessary table expressions to compute for sets, + * and prepared sets. + */ class PlannerActionsVisitor { public: - explicit PlannerActionsVisitor(ActionsDAGPtr actions_dag, const PlannerContextPtr & planner_context_); + explicit PlannerActionsVisitor(const PlannerContextPtr & planner_context_); - ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node); - - ~PlannerActionsVisitor(); + /** Add actions necessary to calculate expression node into expression dag. + * Necessary actions are not added in actions dag output. + * Returns query tree expression node actions dag nodes. + */ + ActionsDAG::NodeRawConstPtrs visit(ActionsDAGPtr actions_dag, QueryTreeNodePtr expression_node); private: - using NodeNameAndNodeMinLevel = std::pair; - - NodeNameAndNodeMinLevel visitImpl(QueryTreeNodePtr node); - - NodeNameAndNodeMinLevel visitColumn(const QueryTreeNodePtr & node); - - NodeNameAndNodeMinLevel visitConstant(const QueryTreeNodePtr & node); - - NodeNameAndNodeMinLevel visitLambda(const QueryTreeNodePtr & node); - - NodeNameAndNodeMinLevel visitFunction(const QueryTreeNodePtr & node); - - String getActionsDAGNodeName(const IQueryTreeNode * node) const; - - class ActionsScopeNode; - std::vector> actions_stack; const PlannerContextPtr planner_context; }; diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index fa50bdf66a4..ec9264a9533 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -7,6 +7,7 @@ #include #include +#include #include @@ -109,41 +110,57 @@ private: ColumnNameToColumnIdentifier column_name_to_column_identifier; }; -class Set; -using SetPtr = std::shared_ptr; +struct SubqueryNodeForSet +{ + QueryTreeNodePtr subquery_node; + SetPtr set; +}; class GlobalPlannerContext { public: GlobalPlannerContext() = default; - void registerSet(UInt128 source_hash, SetPtr set) + using SetKeyToSet = std::unordered_map; + using SetKeyToSubqueryNode = std::unordered_map; + + void registerSet(const String & key, const SetPtr & set) { - set_source_to_set.emplace(source_hash, set); + set_key_to_set.emplace(key, set); } - SetPtr getSet(UInt128 source_hash) const + SetPtr getSet(const String & key) const { - auto it = set_source_to_set.find(source_hash); - if (it == set_source_to_set.end()) + auto it = set_key_to_set.find(key); + if (it == set_key_to_set.end()) return nullptr; return it->second; } - void registerSubqueryForSet(String key, SubqueryForSet subquery_for_set) + void registerSubqueryNodeForSet(const String & key, const SubqueryNodeForSet & subquery_node_for_set) { - subqueries_for_sets.emplace(key, std::move(subquery_for_set)); + auto node_type = subquery_node_for_set.subquery_node->getNodeType(); + if (node_type != QueryTreeNodeType::QUERY && + node_type != QueryTreeNodeType::UNION) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Invalid node for set table expression. Expected query or union. Actual {}", + subquery_node_for_set.subquery_node->formatASTForErrorMessage()); + if (!subquery_node_for_set.set) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Set must be initialized"); + + set_key_to_subquery_node.emplace(key, subquery_node_for_set); } - const SubqueriesForSets & getSubqueriesForSets() const + const SetKeyToSubqueryNode & getSubqueryNodesForSets() const { - return subqueries_for_sets; + return set_key_to_subquery_node; } private: - std::unordered_map set_source_to_set; + SetKeyToSet set_key_to_set; - SubqueriesForSets subqueries_for_sets; + SetKeyToSubqueryNode set_key_to_subquery_node; }; using GlobalPlannerContextPtr = std::shared_ptr; diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 813a8ecb2c6..fb4486a18fe 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -221,8 +221,8 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & { ActionsDAGPtr join_expression_actions = std::make_shared(join_expression_input_columns); - PlannerActionsVisitor join_expression_visitor(join_expression_actions, planner_context); - auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_node.getJoinExpression()); + PlannerActionsVisitor join_expression_visitor(planner_context); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_node.getJoinExpression()); if (join_expression_dag_node_raw_pointers.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index fdd976f1374..8c48e2e56a7 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -1,9 +1,14 @@ #include +#include +#include + #include #include +#include + namespace DB { @@ -56,4 +61,18 @@ Block buildCommonHeaderForUnion(const Blocks & queries_headers) return common_header; } +ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node) +{ + auto & query_node_typed = query_node->as(); + auto result_ast = query_node_typed.toAST(); + + if (auto * select_with_union = result_ast->as()) + result_ast = select_with_union->list_of_selects->children.at(0); + + if (auto * subquery = result_ast->as()) + result_ast = subquery->children.at(0); + + return result_ast; +} + } diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 2e294e0e156..610e5d13d6d 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -1,8 +1,13 @@ #pragma once #include + +#include + #include +#include + namespace DB { @@ -15,4 +20,7 @@ String dumpQueryPipeline(QueryPlan & query_plan); /// Build common header for UNION query Block buildCommonHeaderForUnion(const Blocks & queries_headers); +/// Convert query node to ASTSelectQuery +ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node); + } diff --git a/tests/queries/0_stateless/02376_function_in_function_subquery.reference b/tests/queries/0_stateless/02376_function_in_function_subquery.reference new file mode 100644 index 00000000000..4ae0e409261 --- /dev/null +++ b/tests/queries/0_stateless/02376_function_in_function_subquery.reference @@ -0,0 +1,40 @@ +-- { echoOn } + +SELECT id, value FROM test_table WHERE id IN (SELECT 1); +1 Value_1 +SELECT '--'; +-- +SELECT id, value FROM test_table WHERE id IN (SELECT 2); +2 Value_2 +DROP TABLE IF EXISTS test_table_for_in; +CREATE TABLE test_table_for_in +( + id UInt64 +) ENGINE=TinyLog; +INSERT INTO test_table_for_in VALUES (0), (1); +SELECT '--'; +-- +SELECT id, value FROM test_table WHERE id IN test_table_for_in; +0 Value_0 +1 Value_1 +SELECT '--'; +-- +SELECT id, value FROM test_table WHERE id IN (SELECT id FROM test_table_for_in); +0 Value_0 +1 Value_1 +SELECT '--'; +-- +SELECT id, value FROM test_table WHERE id IN (SELECT id FROM test_table_for_in UNION DISTINCT SELECT id FROM test_table_for_in); +0 Value_0 +1 Value_1 +SELECT '--'; +-- +WITH cte_test_table_for_in AS (SELECT id FROM test_table_for_in) SELECT id, value FROM test_table WHERE id IN cte_test_table_for_in; +0 Value_0 +1 Value_1 +SELECT '--'; +-- +WITH cte_test_table_for_in AS (SELECT id FROM test_table_for_in) SELECT id, value +FROM test_table WHERE id IN (SELECT id FROM cte_test_table_for_in UNION DISTINCT SELECT id FROM cte_test_table_for_in); +0 Value_0 +1 Value_1 diff --git a/tests/queries/0_stateless/02376_function_in_function_subquery.sql b/tests/queries/0_stateless/02376_function_in_function_subquery.sql new file mode 100644 index 00000000000..b199de96534 --- /dev/null +++ b/tests/queries/0_stateless/02376_function_in_function_subquery.sql @@ -0,0 +1,52 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value_0'), (1, 'Value_1'), (2, 'Value_2'); + +-- { echoOn } + +SELECT id, value FROM test_table WHERE id IN (SELECT 1); + +SELECT '--'; + +SELECT id, value FROM test_table WHERE id IN (SELECT 2); + +DROP TABLE IF EXISTS test_table_for_in; +CREATE TABLE test_table_for_in +( + id UInt64 +) ENGINE=TinyLog; + +INSERT INTO test_table_for_in VALUES (0), (1); + +SELECT '--'; + +SELECT id, value FROM test_table WHERE id IN test_table_for_in; + +SELECT '--'; + +SELECT id, value FROM test_table WHERE id IN (SELECT id FROM test_table_for_in); + +SELECT '--'; + +SELECT id, value FROM test_table WHERE id IN (SELECT id FROM test_table_for_in UNION DISTINCT SELECT id FROM test_table_for_in); + +SELECT '--'; + +WITH cte_test_table_for_in AS (SELECT id FROM test_table_for_in) SELECT id, value FROM test_table WHERE id IN cte_test_table_for_in; + +SELECT '--'; + +WITH cte_test_table_for_in AS (SELECT id FROM test_table_for_in) SELECT id, value +FROM test_table WHERE id IN (SELECT id FROM cte_test_table_for_in UNION DISTINCT SELECT id FROM cte_test_table_for_in); + +-- { echoOff } + +DROP TABLE test_table; +DROP TABLE test_table_for_in; From 5c6d175028e5fcbf31669482d35b72c3238ec5b4 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Aug 2022 16:19:35 +0200 Subject: [PATCH 074/188] Added GROUP BY support --- src/Analyzer/FunctionNode.cpp | 1 + src/Analyzer/InDepthQueryTreeVisitor.h | 10 +- src/Analyzer/QueryAnalysisPass.cpp | 12 +- src/Analyzer/QueryNode.cpp | 17 +- src/Analyzer/QueryNode.h | 30 +++ src/Analyzer/QueryTreeBuilder.cpp | 23 +- src/Analyzer/TableFunctionNode.h | 8 + src/Analyzer/Utils.cpp | 8 +- src/Planner/ActionsChain.cpp | 37 +++- src/Planner/ActionsChain.h | 33 ++- src/Planner/Planner.cpp | 277 ++++++++++++++++++++++++- src/Planner/PlannerActionsVisitor.cpp | 52 ++--- src/Planner/PlannerActionsVisitor.h | 15 ++ src/Planner/PlannerContext.cpp | 60 +++--- src/Planner/PlannerContext.h | 6 +- src/Planner/Utils.cpp | 2 + src/Planner/Utils.h | 2 + 17 files changed, 491 insertions(+), 102 deletions(-) diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index 195e9306cdb..54e68acab55 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -38,6 +38,7 @@ void FunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state buffer << ", alias: " << getAlias(); buffer << ", function_name: " << function_name; + buffer << ", function_type: " << (function ? "ordinary_function" : "aggregate_function"); if (result_type) buffer << ", result_type: " + result_type->getName(); diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index 3eba5f2b15c..a1e71113263 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -11,11 +11,12 @@ namespace DB /** Visit query tree in depth. * Matcher need to define `visit`, `needChildVisit` methods and `Data` type. */ -template +template class InDepthQueryTreeVisitor { public: using Data = typename Matcher::Data; + using VisitQueryTreeNodeType = std::conditional_t; /// Initialize visitor with matchers data explicit InDepthQueryTreeVisitor(Data & data_) @@ -23,7 +24,7 @@ public: {} /// Visit query tree node - void visit(QueryTreeNodePtr & query_tree_node) + void visit(VisitQueryTreeNodeType & query_tree_node) { if constexpr (!top_to_bottom) visitChildren(query_tree_node); @@ -45,7 +46,7 @@ public: private: Data & data; - void visitChildren(QueryTreeNodePtr & expression) + void visitChildren(VisitQueryTreeNodeType & expression) { for (auto & child : expression->getChildren()) { @@ -64,4 +65,7 @@ private: } }; +template +using ConstInDepthQueryTreeVisitor = InDepthQueryTreeVisitor; + } diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index fc6eeef5771..f974d724ba5 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -2002,7 +2002,8 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, else { throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unqualified matcher resolve unexpected FROM section {}", + "Unqualified matcher {} resolve unexpected FROM section {}", + matcher_node_typed.formatASTForErrorMessage(), scope_query_node->getJoinTree()->formatASTForErrorMessage()); } @@ -3506,17 +3507,20 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /// Resolve query node sections. - if (query_node_typed.getWithNode()) + if (!query_node_typed.getWith().getNodes().empty()) resolveExpressionNodeList(query_node_typed.getWithNode(), scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); - resolveExpressionNodeList(query_node_typed.getProjectionNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - if (query_node_typed.getPrewhere()) resolveExpressionNode(query_node_typed.getPrewhere(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); if (query_node_typed.getWhere()) resolveExpressionNode(query_node_typed.getWhere(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + if (!query_node_typed.getGroupBy().getNodes().empty()) + resolveExpressionNodeList(query_node_typed.getGroupByNode(), scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); + + resolveExpressionNodeList(query_node_typed.getProjectionNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + /** WITH section can be safely removed, because WITH section only can provide aliases to query expressions * and CTE for other sections to use. * diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 5f754e31d0b..9c006d636c1 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -26,6 +26,7 @@ QueryNode::QueryNode() children.resize(children_size); children[with_child_index] = std::make_shared(); children[projection_child_index] = std::make_shared(); + children[group_by_child_index] = std::make_shared(); } NamesAndTypesList QueryNode::computeProjectionColumns() const @@ -81,6 +82,11 @@ String QueryNode::getName() const buffer << getWhere()->getName(); } + if (!getGroupBy().getNodes().empty()) + { + buffer << getGroupBy().getName(); + } + return buffer.str(); } @@ -124,6 +130,12 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << '\n' << std::string(indent + 2, ' ') << "WHERE\n"; getWhere()->dumpTreeImpl(buffer, format_state, indent + 4); } + + if (!getGroupBy().getNodes().empty()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "GROUP BY\n"; + getGroupBy().dumpTreeImpl(buffer, format_state, indent + 4); + } } bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const @@ -146,7 +158,7 @@ ASTPtr QueryNode::toASTImpl() const auto select_query = std::make_shared(); if (!getWith().getNodes().empty()) - select_query->setExpression(ASTSelectQuery::Expression::WITH, getWithNode()->toAST()); + select_query->setExpression(ASTSelectQuery::Expression::WITH, getWith().toAST()); select_query->setExpression(ASTSelectQuery::Expression::SELECT, children[projection_child_index]->toAST()); @@ -160,6 +172,9 @@ ASTPtr QueryNode::toASTImpl() const if (getWhere()) select_query->setExpression(ASTSelectQuery::Expression::WHERE, getWhere()->toAST()); + if (!getGroupBy().getNodes().empty()) + select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, getGroupBy().toAST()); + auto result_select_query = std::make_shared(); result_select_query->union_mode = SelectUnionMode::Unspecified; diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 911e899ab42..594b0b9b3f7 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -97,6 +97,11 @@ public: return children[join_tree_child_index]; } + bool hasPrewhere() const + { + return children[prewhere_child_index] != nullptr; + } + const QueryTreeNodePtr & getPrewhere() const { return children[prewhere_child_index]; @@ -122,6 +127,31 @@ public: return children[where_child_index]; } + bool hasGroupBy() const + { + return !getGroupBy().getNodes().empty(); + } + + const ListNode & getGroupBy() const + { + return children[group_by_child_index]->as(); + } + + ListNode & getGroupBy() + { + return children[group_by_child_index]->as(); + } + + const QueryTreeNodePtr & getGroupByNode() const + { + return children[group_by_child_index]; + } + + QueryTreeNodePtr & getGroupByNode() + { + return children[group_by_child_index]; + } + /// Compute query node columns using projection section NamesAndTypesList computeProjectionColumns() const; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 9164ac69a0a..277ee07b019 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -196,26 +196,11 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q auto select_with_list = select_query_typed.with(); if (select_with_list) - { - auto & select_with_list_typed = select_with_list->as(); - for (auto & expression_part : select_with_list_typed.children) - { - auto expression_node = buildExpression(expression_part); - current_query_tree->getWith().getNodes().push_back(expression_node); - } - } + current_query_tree->getWithNode() = buildExpressionList(select_with_list); auto select_expression_list = select_query_typed.select(); if (select_expression_list) - { - auto & select_expression_list_typed = select_expression_list->as(); - - for (auto & expression_part : select_expression_list_typed.children) - { - auto expression_node = buildExpression(expression_part); - current_query_tree->getProjection().getNodes().push_back(expression_node); - } - } + current_query_tree->getProjectionNode() = buildExpressionList(select_expression_list); auto prewhere_expression = select_query_typed.prewhere(); if (prewhere_expression) @@ -225,6 +210,10 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q if (where_expression) current_query_tree->getWhere() = buildExpression(where_expression); + auto group_by_list = select_query_typed.groupBy(); + if (group_by_list) + current_query_tree->getGroupByNode() = buildExpressionList(group_by_list); + return current_query_tree; } diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index f664812e6b2..794967870f0 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -81,6 +81,14 @@ public: return storage; } + const StoragePtr & getStorageOrThrow() const + { + if (!storage) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Function node is not resolved"); + + return storage; + } + /// Resolve table function with table_function, storage and context void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context); diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 73c305a5304..03d7f350db0 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -70,14 +70,14 @@ static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expre auto result_table_expression = std::make_shared(); result_table_expression->children.push_back(table_expression_node_ast); - if (node_type == QueryTreeNodeType::QUERY) + if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) result_table_expression->subquery = result_table_expression->children.back(); else if (node_type == QueryTreeNodeType::TABLE || node_type == QueryTreeNodeType::IDENTIFIER) result_table_expression->database_and_table_name = result_table_expression->children.back(); else if (node_type == QueryTreeNodeType::TABLE_FUNCTION) result_table_expression->table_function = result_table_expression->children.back(); else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected identiifer, table, query, or table function. Actual {}", table_expression_node->formatASTForErrorMessage()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected identiifer, table, query, union or table function. Actual {}", table_expression_node->formatASTForErrorMessage()); return result_table_expression; } @@ -94,6 +94,8 @@ void addTableExpressionIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_a [[fallthrough]]; case QueryTreeNodeType::QUERY: [[fallthrough]]; + case QueryTreeNodeType::UNION: + [[fallthrough]]; case QueryTreeNodeType::TABLE_FUNCTION: { auto table_expression_ast = convertIntoTableExpressionAST(table_expression); @@ -118,7 +120,7 @@ void addTableExpressionIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_a default: { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected node type for table expression. Expected table, query, table function, join or array join. Actual {}", + "Unexpected node type for table expression. Expected table, table function, query, union, join or array join. Actual {}", table_expression->getNodeTypeName()); } } diff --git a/src/Planner/ActionsChain.cpp b/src/Planner/ActionsChain.cpp index c7aa8361ed4..89199628187 100644 --- a/src/Planner/ActionsChain.cpp +++ b/src/Planner/ActionsChain.cpp @@ -40,6 +40,13 @@ void ActionsChainStep::dump(WriteBuffer & buffer) const { buffer << "DAG" << '\n'; buffer << actions->dumpDAG(); + + if (!additional_output_columns.empty()) + { + buffer << "Additional output columns " << additional_output_columns.size() << '\n'; + for (const auto & column : additional_output_columns) + buffer << "Name " << column.name << " type " << column.type->getName() << '\n'; + } if (!child_required_output_columns_names.empty()) { buffer << "Child required output columns " << boost::join(child_required_output_columns_names, ", "); @@ -62,22 +69,30 @@ void ActionsChainStep::initialize() available_output_columns.clear(); - for (const auto & node : actions->getNodes()) + if (available_output_columns_strategy == AvailableOutputColumnsStrategy::ALL_NODES) { - if (available_output_columns_only_aliases) + for (const auto & node : actions->getNodes()) { - if (node.type == ActionsDAG::ActionType::ALIAS) + if (node.type == ActionsDAG::ActionType::INPUT || + node.type == ActionsDAG::ActionType::FUNCTION || + node.type == ActionsDAG::ActionType::ALIAS || + node.type == ActionsDAG::ActionType::ARRAY_JOIN) available_output_columns.emplace_back(node.column, node.result_type, node.result_name); - - continue; } - - if (node.type == ActionsDAG::ActionType::INPUT || - node.type == ActionsDAG::ActionType::FUNCTION || - node.type == ActionsDAG::ActionType::ALIAS || - node.type == ActionsDAG::ActionType::ARRAY_JOIN) - available_output_columns.emplace_back(node.column, node.result_type, node.result_name); } + else if (available_output_columns_strategy == AvailableOutputColumnsStrategy::OUTPUT_NODES) + { + for (const auto & node : actions->getOutputs()) + { + if (node->type == ActionsDAG::ActionType::INPUT || + node->type == ActionsDAG::ActionType::FUNCTION || + node->type == ActionsDAG::ActionType::ALIAS || + node->type == ActionsDAG::ActionType::ARRAY_JOIN) + available_output_columns.emplace_back(node->column, node->result_type, node->result_name); + } + } + + available_output_columns.insert(available_output_columns.end(), additional_output_columns.begin(), additional_output_columns.end()); } void ActionsChain::finalize() diff --git a/src/Planner/ActionsChain.h b/src/Planner/ActionsChain.h index c98e8b52e17..d9c168c1d96 100644 --- a/src/Planner/ActionsChain.h +++ b/src/Planner/ActionsChain.h @@ -43,15 +43,34 @@ using ActionsChainSteps = std::vector; class ActionsChainStep { public: + /// Available output columns strategy for actions chain step + enum class AvailableOutputColumnsStrategy + { + ALL_NODES, + OUTPUT_NODES + }; + /** Initialize actions step with actions dag. * Input column names initialized using actions dag nodes with INPUT type. - * Avaiable output columns initialized using actions dag output nodes with INPUT, FUNCTION, ALIAS, ARRAY_JOIN types. - * If available_output_columns_only_aliases is set to true, then only aliases from actions dag will be used - * as available output columns. + * + * If available output columns strategy is ALL_NODES, then avaiable output columns initialized using actions dag nodes + * with INPUT, FUNCTION, ALIAS, ARRAY_JOIN types. + * If available output columns strategy is OUTPUT_NODES, then available output columns initialized using actions dag output nodes + * with INPUT, FUNCTION, ALIAS, ARRAY_JOIN types. */ - explicit ActionsChainStep(ActionsDAGPtr actions_, bool available_output_columns_only_aliases_ = false) + explicit ActionsChainStep(ActionsDAGPtr actions_, AvailableOutputColumnsStrategy available_output_columns_stategy_ = AvailableOutputColumnsStrategy::ALL_NODES) : actions(std::move(actions_)) - , available_output_columns_only_aliases(available_output_columns_only_aliases_) + , available_output_columns_strategy(available_output_columns_stategy_) + { + initialize(); + } + + explicit ActionsChainStep(ActionsDAGPtr actions_, + AvailableOutputColumnsStrategy available_output_columns_stategy_, + const ColumnsWithTypeAndName & additional_output_columns_) + : actions(std::move(actions_)) + , available_output_columns_strategy(available_output_columns_stategy_) + , additional_output_columns(additional_output_columns_) { initialize(); } @@ -104,13 +123,15 @@ private: ActionsDAGPtr actions; - bool available_output_columns_only_aliases; + AvailableOutputColumnsStrategy available_output_columns_strategy; NameSet input_columns_names; NameSet child_required_output_columns_names; ColumnsWithTypeAndName available_output_columns; + + ColumnsWithTypeAndName additional_output_columns; }; /// Query actions chain diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 0ff2ad75442..8fc329c0744 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -62,6 +63,8 @@ namespace ErrorCodes extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; extern const int INVALID_JOIN_ON_EXPRESSION; + extern const int ILLEGAL_AGGREGATION; + extern const int NOT_AN_AGGREGATE; } /** ClickHouse query planner. @@ -82,11 +85,156 @@ namespace ErrorCodes * TODO: Support read in order optimization * TODO: UNION storage limits * TODO: Interpreter resources + * TODO: Support max streams + * TODO: Support GROUPINS SETS, const aggregation keys, overflow row + * TODO: Simplify buildings SETS for IN function */ namespace { +class CollectAggregateFunctionNodesMatcher +{ +public: + using Visitor = ConstInDepthQueryTreeVisitor; + + struct Data + { + Data(String assert_no_aggregates_place_message_, const PlannerContext & planner_context_) + : assert_no_aggregates_place_message(std::move(assert_no_aggregates_place_message_)) + , planner_context(planner_context_) + {} + + explicit Data(const PlannerContext & planner_context_) + : planner_context(planner_context_) + {} + + String assert_no_aggregates_place_message; + const PlannerContext & planner_context; + QueryTreeNodes aggregate_function_nodes; + }; + + static void visit(const QueryTreeNodePtr & node, Data & data) + { + auto * function_node = node->as(); + if (!function_node) + return; + + if (!function_node->isAggregateFunction()) + return; + + if (!data.assert_no_aggregates_place_message.empty()) + throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, + "Aggregate function {} is found {} in query", + function_node->getName(), + data.assert_no_aggregates_place_message); + + data.aggregate_function_nodes.push_back(node); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) + { + return child_node->getNodeType() != QueryTreeNodeType::QUERY; + } +}; + +using CollectAggregateFunctionNodesVisitor = CollectAggregateFunctionNodesMatcher::Visitor; + +void assertNoAggregatesFunctions(const QueryTreeNodePtr & node, const String & assert_no_aggregates_place_message, const PlannerContext & planner_context) +{ + CollectAggregateFunctionNodesVisitor::Data data(assert_no_aggregates_place_message, planner_context); + CollectAggregateFunctionNodesVisitor(data).visit(node); +} + +AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodePtr & query_tree, const PlannerContext & planner_context) +{ + CollectAggregateFunctionNodesVisitor::Data data(planner_context); + CollectAggregateFunctionNodesVisitor(data).visit(query_tree); + + QueryTreeNodeToName node_to_name; + NameSet unique_aggregate_action_node_names; + AggregateDescriptions aggregate_descriptions; + + for (auto & aggregate_function_node : data.aggregate_function_nodes) + { + for (auto & aggregate_function_node_child : aggregate_function_node->getChildren()) + assertNoAggregatesFunctions(aggregate_function_node_child, "inside another aggregate function", planner_context); + + auto & aggregagte_function_node_typed = aggregate_function_node->as(); + String node_name = calculateActionsDAGNodeName(aggregate_function_node.get(), data.planner_context, node_to_name); + auto [_, inserted] = unique_aggregate_action_node_names.emplace(node_name); + if (!inserted) + continue; + + AggregateDescription aggregate_description; + aggregate_description.function = aggregagte_function_node_typed.getAggregateFunction(); + + const auto & parameters_nodes = aggregagte_function_node_typed.getParameters().getNodes(); + aggregate_description.parameters.reserve(parameters_nodes.size()); + + for (const auto & parameter_node : parameters_nodes) + { + const auto & constant_node = parameter_node->as(); + aggregate_description.parameters.push_back(constant_node.getConstantValue()); + } + + const auto & arguments_nodes = aggregagte_function_node_typed.getArguments().getNodes(); + aggregate_description.argument_names.reserve(arguments_nodes.size()); + + for (const auto & argument_node : arguments_nodes) + { + String argument_node_name = calculateActionsDAGNodeName(argument_node.get(), data.planner_context, node_to_name); + aggregate_description.argument_names.emplace_back(std::move(argument_node_name)); + } + + aggregate_description.column_name = node_name; + aggregate_descriptions.push_back(std::move(aggregate_description)); + } + + return aggregate_descriptions; +} + +class ValidateGroupByColumnsMatcher +{ +public: + using Visitor = ConstInDepthQueryTreeVisitor; + + struct Data + { + Data(const NameSet & group_by_keys_column_names_, const PlannerContext & planner_context_) + : group_by_keys_column_names(group_by_keys_column_names_) + , planner_context(planner_context_) + {} + + const NameSet & group_by_keys_column_names; + const PlannerContext & planner_context; + }; + + static void visit(const QueryTreeNodePtr & node, Data & data) + { + auto * column_node = node->as(); + if (!column_node) + return; + + String column_node_name = calculateActionsDAGNodeName(node.get(), data.planner_context); + if (!data.group_by_keys_column_names.contains(column_node_name)) + throw Exception(ErrorCodes::NOT_AN_AGGREGATE, + "Column {} is not under aggregate function and not in GROUP BY keys", + column_node->formatASTForErrorMessage()); + } + + static bool needChildVisit(const QueryTreeNodePtr & parent_node, const QueryTreeNodePtr & child_node) + { + auto * function_node = parent_node->as(); + if (function_node && function_node->isAggregateFunction()) + return false; + + return child_node->getNodeType() != QueryTreeNodeType::QUERY; + } +}; + +using ValidateGroupByColumnsVisitor = typename ValidateGroupByColumnsMatcher::Visitor; + class CollectTableExpressionIdentifiersVisitor { public: @@ -871,6 +1019,46 @@ void Planner::buildQueryPlanIfNeeded() where_action_step_index = actions_chain.getLastStepIndex(); } + if (query_node.hasWhere()) + assertNoAggregatesFunctions(query_node.getWhere(), "in WHERE", *planner_context); + if (query_node.hasPrewhere()) + assertNoAggregatesFunctions(query_node.getWhere(), "in PREWHERE", *planner_context); + + AggregateDescriptions aggregates_descriptions = extractAggregateDescriptions(query_tree, *planner_context); + ColumnsWithTypeAndName aggregates_columns; + aggregates_columns.reserve(aggregates_descriptions.size()); + for (auto & aggregate_description : aggregates_descriptions) + aggregates_columns.emplace_back(nullptr, aggregate_description.function->getReturnType(), aggregate_description.column_name); + + NameSet aggregate_keys_set; + Names aggregate_keys; + std::optional aggregate_step_index; + + if (query_node.hasGroupBy()) + { + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & group_by_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + /// Only aggregation keys, and aggregates are available for next steps after GROUP BY step + auto group_by_actions = convertExpressionNodeIntoDAG(query_node.getGroupByNode(), group_by_input, planner_context); + + aggregate_keys.reserve(group_by_actions->getOutputs().size()); + for (auto & output : group_by_actions->getOutputs()) + { + aggregate_keys_set.insert(output->result_name); + aggregate_keys.push_back(output->result_name); + } + + ValidateGroupByColumnsVisitor::Data validate_group_by_visitor_data(aggregate_keys_set, *planner_context); + ValidateGroupByColumnsVisitor validate_columns_visitor(validate_group_by_visitor_data); + + validate_columns_visitor.visit(query_node.getProjectionNode()); + + auto aggregate_step = std::make_unique(std::move(group_by_actions), ActionsChainStep::AvailableOutputColumnsStrategy::OUTPUT_NODES, aggregates_columns); + actions_chain.addStep(std::move(aggregate_step)); + aggregate_step_index = actions_chain.getLastStepIndex(); + } + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); auto projection_actions = convertExpressionNodeIntoDAG(query_node.getProjectionNode(), projection_input, planner_context); @@ -913,7 +1101,7 @@ void Planner::buildQueryPlanIfNeeded() actions_chain.finalize(); // std::cout << "Chain dump after finalize" << std::endl; - // std::cout << planner_context.actions_chain.dump() << std::endl; + // std::cout << actions_chain.dump() << std::endl; if (where_action_step_index) { @@ -927,6 +1115,93 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(where_step)); } + if (!aggregates_descriptions.empty() || query_node.hasGroupBy()) + { + if (aggregate_step_index) + { + auto & aggregate_actions_chain_node = actions_chain.at(*aggregate_step_index); + auto expression_before_aggregation = std::make_unique(query_plan.getCurrentDataStream(), + aggregate_actions_chain_node->getActions()); + expression_before_aggregation->setStepDescription("Before GROUP BY"); + query_plan.addStep(std::move(expression_before_aggregation)); + } + + const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); + + bool overflow_row = false; + bool query_analyzer_const_aggregation_keys = false; + + const auto stats_collecting_params = Aggregator::Params::StatsCollectingParams( + select_query_info.query, + settings.collect_hash_table_stats_during_aggregation, + settings.max_entries_for_hash_table_stats, + settings.max_size_to_preallocate_for_aggregation); + + Aggregator::Params aggregator_params = Aggregator::Params( + aggregate_keys, + aggregates_descriptions, + overflow_row, + settings.max_rows_to_group_by, + settings.group_by_overflow_mode, + settings.group_by_two_level_threshold, + settings.group_by_two_level_threshold_bytes, + settings.max_bytes_before_external_group_by, + settings.empty_result_for_aggregation_by_empty_set + || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && aggregate_keys.empty() + && query_analyzer_const_aggregation_keys), + planner_context->getQueryContext()->getTemporaryVolume(), + settings.max_threads, + settings.min_free_disk_space_for_temporary_data, + settings.compile_aggregate_expressions, + settings.min_count_to_compile_aggregate_expression, + /* only_merge */ false, + stats_collecting_params + ); + + GroupingSetsParamsList grouping_sets_params; + SortDescription group_by_sort_description; + + auto merge_threads = settings.max_threads; + auto temporary_data_merge_threads = settings.aggregation_memory_efficient_merge_threads + ? static_cast(settings.aggregation_memory_efficient_merge_threads) + : static_cast(settings.max_threads); + + bool storage_has_evenly_distributed_read = false; + auto & table_expression_node_to_columns = planner_context->getTableExpressionNodeToColumns(); + if (table_expression_node_to_columns.size() == 1) + { + auto it = table_expression_node_to_columns.begin(); + const auto * table_expression_node = it->first; + if (const auto * table_node = table_expression_node->as()) + storage_has_evenly_distributed_read = table_node->getStorage()->hasEvenlyDistributedRead(); + else if (const auto * table_function_node = table_expression_node->as()) + storage_has_evenly_distributed_read = table_function_node->getStorageOrThrow()->hasEvenlyDistributedRead(); + } + + const bool should_produce_results_in_order_of_bucket_number + = select_query_options.to_stage == QueryProcessingStage::WithMergeableState && settings.distributed_aggregation_memory_efficient; + + bool aggregate_final = select_query_options.to_stage > QueryProcessingStage::WithMergeableState; + + InputOrderInfoPtr input_order_info; + + auto aggregating_step = std::make_unique( + query_plan.getCurrentDataStream(), + std::move(aggregator_params), + std::move(grouping_sets_params), + aggregate_final, + settings.max_block_size, + settings.aggregation_in_order_max_block_bytes, + merge_threads, + temporary_data_merge_threads, + storage_has_evenly_distributed_read, + settings.group_by_use_nulls, + std::move(input_order_info), + std::move(group_by_sort_description), + should_produce_results_in_order_of_bucket_number); + query_plan.addStep(std::move(aggregating_step)); + } + // std::cout << "Query plan dump" << std::endl; // std::cout << dumpQueryPlan(query_plan) << std::endl; diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 79241091f03..b320fed2803 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -19,6 +19,7 @@ #include #include +#include namespace DB { @@ -164,8 +165,6 @@ private: NodeNameAndNodeMinLevel visitFunction(const QueryTreeNodePtr & node); - String calculateActionsDAGNodeName(const IQueryTreeNode * node); - std::vector actions_stack; std::unordered_map node_to_node_name; const PlannerContextPtr planner_context; @@ -214,7 +213,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitColumn(const QueryTreeNodePtr & node) { - auto column_node_name = calculateActionsDAGNodeName(node.get()); + auto column_node_name = calculateActionsDAGNodeName(node.get(), *planner_context, node_to_node_name); const auto & column_node = node->as(); Int64 actions_stack_size = static_cast(actions_stack.size() - 1); @@ -234,7 +233,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitConstant(const QueryTreeNodePtr & node) { - auto constant_node_name = calculateActionsDAGNodeName(node.get()); + auto constant_node_name = calculateActionsDAGNodeName(node.get(), *planner_context, node_to_node_name); const auto & constant_node = node->as(); const auto & literal = constant_node.getConstantValue(); @@ -303,7 +302,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi } } - auto lambda_node_name = calculateActionsDAGNodeName(node.get()); + auto lambda_node_name = calculateActionsDAGNodeName(node.get(), *planner_context); auto function_capture = std::make_shared( lambda_actions, captured_column_names, lambda_arguments_names_and_types, result_type, lambda_expression_node_name); actions_stack.pop_back(); @@ -450,7 +449,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi level = std::max(level, node_min_level); } - auto function_node_name = calculateActionsDAGNodeName(node.get()); + auto function_node_name = calculateActionsDAGNodeName(node.get(), *planner_context, node_to_node_name); if (function_node.isAggregateFunction()) { @@ -495,11 +494,22 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi return {function_node_name, level}; } -String PlannerActionsVisitorImpl::calculateActionsDAGNodeName(const IQueryTreeNode * node) -{ - auto it = node_to_node_name.find(node); +} - if (it != node_to_node_name.end()) +PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_context_) + : planner_context(planner_context_) +{} + +ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAGPtr actions_dag, QueryTreeNodePtr expression_node) +{ + PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context); + return actions_visitor_impl.visit(expression_node); +} + +String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name) +{ + auto it = node_to_name.find(node); + if (it != node_to_name.end()) return it->second; String result; @@ -509,7 +519,7 @@ String PlannerActionsVisitorImpl::calculateActionsDAGNodeName(const IQueryTreeNo { case QueryTreeNodeType::COLUMN: { - const auto * column_identifier = planner_context->getColumnNodeIdentifierOrNull(node); + const auto * column_identifier = planner_context.getColumnNodeIdentifierOrNull(node); result = column_identifier ? *column_identifier : node->getName(); break; @@ -537,7 +547,7 @@ String PlannerActionsVisitorImpl::calculateActionsDAGNodeName(const IQueryTreeNo for (size_t i = 0; i < function_parameters_nodes_size; ++i) { const auto & function_parameter_node = function_parameters_nodes[i]; - calculateActionsDAGNodeName(function_parameter_node.get()); + calculateActionsDAGNodeName(function_parameter_node.get(), planner_context); if (i + 1 != function_parameters_nodes_size) buffer << ", "; @@ -554,7 +564,7 @@ String PlannerActionsVisitorImpl::calculateActionsDAGNodeName(const IQueryTreeNo for (size_t i = 0; i < function_arguments_nodes_size; ++i) { const auto & function_argument_node = function_arguments_nodes[i]; - buffer << calculateActionsDAGNodeName(function_argument_node.get()); + buffer << calculateActionsDAGNodeName(function_argument_node.get(), planner_context); if (i + 1 != function_arguments_nodes_size) buffer << ", "; @@ -582,26 +592,20 @@ String PlannerActionsVisitorImpl::calculateActionsDAGNodeName(const IQueryTreeNo default: { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Actions visitor invalid query tree node {}", + "Invalid action query tree node {}", node->formatASTForErrorMessage()); } } - node_to_node_name.emplace(node, result); + node_to_name.emplace(node, result); return result; } -} - -PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_context_) - : planner_context(planner_context_) -{} - -ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAGPtr actions_dag, QueryTreeNodePtr expression_node) +String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerContext & planner_context) { - PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context); - return actions_visitor_impl.visit(expression_node); + QueryTreeNodeToName empty_map; + return calculateActionsDAGNodeName(node, planner_context, empty_map); } } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 13cdca36d5b..94cd98fa87a 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -44,4 +44,19 @@ private: const PlannerContextPtr planner_context; }; +/** Calculate query tree expression node name action dag name and add them into node to name map. + * If node exists in map, name from map is used. + * + * For column node column node identifier from planner context is used. + */ +using QueryTreeNodeToName = std::unordered_map; +String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name); + +/** Calculate query tree expression node name action dag name. + * + * For column node column node identifier from planner context is used. + */ +String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerContext & planner_context); + + } diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index df179cd544f..37e208ce8f8 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -15,36 +15,6 @@ PlannerContext::PlannerContext(ContextPtr query_context_, GlobalPlannerContextPt , global_planner_context(std::move(global_planner_context_)) {} -void PlannerContext::registerColumnNode(const IQueryTreeNode * column_node, const ColumnIdentifier & column_identifier) -{ - assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); - column_node_to_column_identifier.emplace(column_node, column_identifier); -} - -const ColumnIdentifier & PlannerContext::getColumnNodeIdentifierOrThrow(const IQueryTreeNode * column_node) -{ - assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); - - auto it = column_node_to_column_identifier.find(column_node); - if (it == column_node_to_column_identifier.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Column identifier is not initialized for column {}", - column_node->formatASTForErrorMessage()); - - return it->second; -} - -const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const IQueryTreeNode * column_node) -{ - assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); - - auto it = column_node_to_column_identifier.find(column_node); - if (it == column_node_to_column_identifier.end()) - return nullptr; - - return &it->second; -} - ColumnIdentifier PlannerContext::getColumnUniqueIdentifier(const IQueryTreeNode * column_source_node, std::string column_name) { auto column_unique_prefix = "__column_" + std::to_string(column_identifier_counter); @@ -90,4 +60,34 @@ ColumnIdentifier PlannerContext::getColumnUniqueIdentifier(const IQueryTreeNode return column_unique_prefix; } +void PlannerContext::registerColumnNode(const IQueryTreeNode * column_node, const ColumnIdentifier & column_identifier) +{ + assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); + column_node_to_column_identifier.emplace(column_node, column_identifier); +} + +const ColumnIdentifier & PlannerContext::getColumnNodeIdentifierOrThrow(const IQueryTreeNode * column_node) const +{ + assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); + + auto it = column_node_to_column_identifier.find(column_node); + if (it == column_node_to_column_identifier.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column identifier is not initialized for column {}", + column_node->formatASTForErrorMessage()); + + return it->second; +} + +const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const IQueryTreeNode * column_node) const +{ + assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); + + auto it = column_node_to_column_identifier.find(column_node); + if (it == column_node_to_column_identifier.end()) + return nullptr; + + return &it->second; +} + } diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index ec9264a9533..c85a37a1570 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -209,9 +209,9 @@ public: void registerColumnNode(const IQueryTreeNode * column_node, const ColumnIdentifier & column_identifier); - const ColumnIdentifier & getColumnNodeIdentifierOrThrow(const IQueryTreeNode * column_node); + const ColumnIdentifier & getColumnNodeIdentifierOrThrow(const IQueryTreeNode * column_node) const; - const ColumnIdentifier * getColumnNodeIdentifierOrNull(const IQueryTreeNode * column_node); + const ColumnIdentifier * getColumnNodeIdentifierOrNull(const IQueryTreeNode * column_node) const; private: /// Query context @@ -232,4 +232,6 @@ private: size_t column_identifier_counter = 0; }; +using PlannerContextPtr = std::shared_ptr; + } diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 8c48e2e56a7..7392aa8b9ff 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -8,6 +8,8 @@ #include #include +#include +#include namespace DB { diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 610e5d13d6d..4902aa45e4a 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -8,6 +8,8 @@ #include +#include + namespace DB { From 03864790ad8d4f53324ff85282ff066ba4d16ade Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Aug 2022 20:35:16 +0200 Subject: [PATCH 075/188] Added DISTINCT support --- src/Analyzer/IQueryTreeNode.cpp | 2 +- src/Analyzer/QueryAnalysisPass.cpp | 6 +- src/Analyzer/QueryNode.cpp | 6 +- src/Analyzer/QueryNode.h | 11 +++ src/Analyzer/QueryTreeBuilder.cpp | 14 ++- src/Interpreters/InterpreterExplainQuery.cpp | 1 + src/Planner/Planner.cpp | 90 +++++++++++++++++-- src/Planner/Utils.cpp | 18 +++- .../TableFunctionViewIfPermitted.cpp | 10 ++- 9 files changed, 141 insertions(+), 17 deletions(-) diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 3135ff7f96a..a69afa2eb4f 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -194,7 +194,7 @@ ASTPtr IQueryTreeNode::toAST() const { auto converted_node = toASTImpl(); - if (auto * ast_with_alias = typeid_cast(converted_node.get())) + if (auto * ast_with_alias = dynamic_cast(converted_node.get())) converted_node->setAlias(alias); return converted_node; diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index f974d724ba5..93acfc98907 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -3252,7 +3252,11 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, scope.scope_node->formatASTForErrorMessage()); } - resolveExpressionNode(argument_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + auto * function_node = argument_node->as(); + if (function_node && table_function_factory.hasNameOrAlias(function_node->getFunctionName())) + continue; + + resolveExpressionNode(argument_node, scope, false /*allow_lambda_expression*/, true /*allow_table_expression*/); } auto table_function_ast = table_function_node.toAST(); diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 9c006d636c1..35bee05b85e 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -99,6 +99,7 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << ", is_subquery: " << is_subquery; buffer << ", is_cte: " << is_cte; + buffer << ", is_distinct: " << is_distinct; if (!cte_name.empty()) buffer << ", cte_name: " << cte_name; @@ -141,7 +142,7 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); - return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name; + return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name && is_distinct == rhs_typed.is_distinct; } void QueryNode::updateTreeHashImpl(HashState & state) const @@ -151,11 +152,14 @@ void QueryNode::updateTreeHashImpl(HashState & state) const state.update(cte_name.size()); state.update(cte_name); + + state.update(is_distinct); } ASTPtr QueryNode::toASTImpl() const { auto select_query = std::make_shared(); + select_query->distinct = is_distinct; if (!getWith().getNodes().empty()) select_query->setExpression(ASTSelectQuery::Expression::WITH, getWith().toAST()); diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 594b0b9b3f7..12684a8d059 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -47,6 +47,16 @@ public: cte_name = std::move(cte_name_value); } + void setIsDistinct(bool is_distinct_value) + { + is_distinct = is_distinct_value; + } + + bool isDistinct() const + { + return is_distinct; + } + const ListNode & getWith() const { return children[with_child_index]->as(); @@ -176,6 +186,7 @@ protected: private: bool is_subquery = false; bool is_cte = false; + bool is_distinct = false; std::string cte_name; static constexpr size_t with_child_index = 0; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 277ee07b019..45769d3029d 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -116,7 +116,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectOrUnionExpression(const ASTPtr & s else if (select_or_union_query->as()) query_node = buildSelectExpression(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/); else - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "UNION query {} is not supported", select_or_union_query->formatForErrorMessage()); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "SELECT or UNION query {} is not supported", select_or_union_query->formatForErrorMessage()); return query_node; } @@ -190,6 +190,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q current_query_tree->setIsSubquery(is_subquery); current_query_tree->setIsCTE(!cte_name.empty()); current_query_tree->setCTEName(cte_name); + current_query_tree->setIsDistinct(select_query_typed.distinct); current_query_tree->getJoinTree() = buildJoinTree(select_query_typed.tables()); current_query_tree->setOriginalAST(select_query); @@ -386,7 +387,9 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co } else { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only literals and constants are supported as expression. Actual {}", expression->formatForErrorMessage()); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Invalid expression. Expected identifier, literal, matcher, function, subquery. Actual {}", + expression->formatForErrorMessage()); } result->setAlias(expression->tryGetAlias()); @@ -451,7 +454,12 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select { const auto & function_arguments_list = table_function_expression.arguments->as()->children; for (const auto & argument : function_arguments_list) - node->getArguments().getNodes().push_back(buildExpression(argument)); + { + if (argument->as() || argument->as() || argument->as()) + node->getArguments().getNodes().push_back(buildSelectOrUnionExpression(argument, true /*is_subquery*/, {} /*cte_name*/)); + else + node->getArguments().getNodes().push_back(buildExpression(argument)); + } } node->setAlias(table_function_expression.tryGetAlias()); diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 7428d69bdee..f3580902c3b 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -404,6 +404,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() if (settings.dump_ast) { + buf << '\n'; buf << '\n'; query_tree->toAST()->format(IAST::FormatSettings(buf, false)); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 8fc329c0744..3ddd45ad67c 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -93,6 +93,53 @@ namespace ErrorCodes namespace { +StreamLocalLimits getLimitsForStorage(const Settings & settings, const SelectQueryOptions & options) +{ + StreamLocalLimits limits; + limits.mode = LimitsMode::LIMITS_TOTAL; + limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read, settings.read_overflow_mode); + limits.speed_limits.max_execution_time = settings.max_execution_time; + limits.timeout_overflow_mode = settings.timeout_overflow_mode; + + /** Quota and minimal speed restrictions are checked on the initiating server of the request, and not on remote servers, + * because the initiating server has a summary of the execution of the request on all servers. + * + * But limits on data size to read and maximum execution time are reasonable to check both on initiator and + * additionally on each remote server, because these limits are checked per block of data processed, + * and remote servers may process way more blocks of data than are received by initiator. + * + * The limits to throttle maximum execution speed is also checked on all servers. + */ + if (options.to_stage == QueryProcessingStage::Complete) + { + limits.speed_limits.min_execution_rps = settings.min_execution_speed; + limits.speed_limits.min_execution_bps = settings.min_execution_speed_bytes; + } + + limits.speed_limits.max_execution_rps = settings.max_execution_speed; + limits.speed_limits.max_execution_bps = settings.max_execution_speed_bytes; + limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; + + return limits; +} + +StorageLimits getStorageLimits(const Context & context, const SelectQueryOptions & options) +{ + const auto & settings = context.getSettingsRef(); + + StreamLocalLimits limits; + SizeLimits leaf_limits; + + /// Set the limits and quota for reading data, the speed and time of the query. + if (!options.ignore_limits) + { + limits = getLimitsForStorage(settings, options); + leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, settings.max_bytes_to_read_leaf, settings.read_overflow_mode_leaf); + } + + return {limits, leaf_limits}; +} + class CollectAggregateFunctionNodesMatcher { public: @@ -995,6 +1042,10 @@ void Planner::buildQueryPlanIfNeeded() select_query_info.original_query = queryNodeToSelectQuery(query_tree); select_query_info.query = select_query_info.original_query; + StorageLimitsList storage_limits; + storage_limits.push_back(getStorageLimits(*current_context, select_query_options)); + select_query_info.storage_limits = std::make_shared(storage_limits); + CollectTableExpressionIdentifiersVisitor collect_table_expression_identifiers_visitor; collect_table_expression_identifiers_visitor.visit(query_node.getJoinTree(), *planner_context); @@ -1075,7 +1126,11 @@ void Planner::buildQueryPlanIfNeeded() projection_action_dag_nodes_size, projection_nodes_size); - NamesWithAliases projection_names; + Names projection_names; + projection_names.reserve(projection_nodes_size); + + NamesWithAliases projection_names_with_aliases; + projection_names_with_aliases.reserve(projection_nodes_size); for (size_t i = 0; i < projection_nodes_size; ++i) { @@ -1084,13 +1139,15 @@ void Planner::buildQueryPlanIfNeeded() const auto * action_dag_node = projection_action_dag_nodes[i]; const auto & actions_dag_node_name = action_dag_node->result_name; + projection_names.push_back(actions_dag_node_name); + if (node->hasAlias()) - projection_names.push_back({actions_dag_node_name, node->getAlias()}); + projection_names_with_aliases.push_back({actions_dag_node_name, node->getAlias()}); else - projection_names.push_back({actions_dag_node_name, node_name}); + projection_names_with_aliases.push_back({actions_dag_node_name, node_name}); } - projection_actions->project(projection_names); + projection_actions->project(projection_names_with_aliases); actions_chain.addStep(std::make_unique(std::move(projection_actions))); size_t projection_action_step_index = actions_chain.getLastStepIndex(); @@ -1202,8 +1259,29 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(aggregating_step)); } - // std::cout << "Query plan dump" << std::endl; - // std::cout << dumpQueryPlan(query_plan) << std::endl; + if (query_node.isDistinct()) + { + const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); + UInt64 limit_hint_for_distinct = 0; + bool pre_distinct = true; + + SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); + + auto distinct_step = std::make_unique( + query_plan.getCurrentDataStream(), + limits, + limit_hint_for_distinct, + projection_names, + pre_distinct, + settings.optimize_distinct_in_order); + + if (pre_distinct) + distinct_step->setStepDescription("Preliminary DISTINCT"); + else + distinct_step->setStepDescription("DISTINCT"); + + query_plan.addStep(std::move(distinct_step)); + } auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), actions_chain[projection_action_step_index]->getActions()); projection_step->setStepDescription("Projection"); diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 7392aa8b9ff..96a631612e7 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include @@ -68,11 +69,20 @@ ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node) auto & query_node_typed = query_node->as(); auto result_ast = query_node_typed.toAST(); - if (auto * select_with_union = result_ast->as()) - result_ast = select_with_union->list_of_selects->children.at(0); + while (true) + { + if (auto * select_query = result_ast->as()) + break; + else if (auto * select_with_union = result_ast->as()) + result_ast = select_with_union->list_of_selects->children.at(0); + else if (auto * subquery = result_ast->as()) + result_ast = subquery->children.at(0); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query node invalid conversion to select query"); + } - if (auto * subquery = result_ast->as()) - result_ast = subquery->children.at(0); + if (result_ast == nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query node invalid conversion to select query"); return result_ast; } diff --git a/src/TableFunctions/TableFunctionViewIfPermitted.cpp b/src/TableFunctions/TableFunctionViewIfPermitted.cpp index ba3d2cb9d16..176db0915d4 100644 --- a/src/TableFunctions/TableFunctionViewIfPermitted.cpp +++ b/src/TableFunctions/TableFunctionViewIfPermitted.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -38,7 +39,14 @@ void TableFunctionViewIfPermitted::parseArguments(const ASTPtr & ast_function, C getName()); const auto & arguments = function->arguments->children; - auto * select = arguments[0]->as(); + auto select_argument = arguments[0]; + auto * subquery = arguments[0]->as(); + + if (subquery) + select_argument = subquery->children[0]; + + auto * select = select_argument->as(); + if (!select) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}' requires a SELECT query as its first argument", getName()); create.set(create.select, select->clone()); From 6aa166f36618ac85ed7754d0c60c4f7ce562b566 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Aug 2022 15:06:14 +0200 Subject: [PATCH 076/188] Added ORDER BY support --- src/Analyzer/FunctionNode.cpp | 8 + src/Analyzer/FunctionNode.h | 8 +- src/Analyzer/IQueryTreeNode.cpp | 1 + src/Analyzer/IQueryTreeNode.h | 1 + src/Analyzer/QueryAnalysisPass.cpp | 85 +++++++++- src/Analyzer/QueryNode.cpp | 31 ++-- src/Analyzer/QueryNode.h | 30 ++++ src/Analyzer/QueryTreeBuilder.cpp | 47 +++++- src/Analyzer/SortColumnNode.cpp | 158 ++++++++++++++++++ src/Analyzer/SortColumnNode.h | 152 +++++++++++++++++ src/Planner/Planner.cpp | 94 ++++++++++- src/Planner/PlannerSorting.cpp | 155 +++++++++++++++++ src/Planner/PlannerSorting.h | 14 ++ .../TableFunctionViewIfPermitted.cpp | 10 +- 14 files changed, 755 insertions(+), 39 deletions(-) create mode 100644 src/Analyzer/SortColumnNode.cpp create mode 100644 src/Analyzer/SortColumnNode.h create mode 100644 src/Planner/PlannerSorting.cpp create mode 100644 src/Planner/PlannerSorting.h diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index 54e68acab55..06f0f248847 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -14,6 +14,14 @@ namespace DB { +FunctionNode::FunctionNode(String function_name_) + : function_name(function_name_) +{ + children.resize(2); + children[parameters_child_index] = std::make_shared(); + children[arguments_child_index] = std::make_shared(); +} + void FunctionNode::resolveAsFunction(FunctionOverloadResolverPtr function_value, DataTypePtr result_type_value) { aggregate_function = nullptr; diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index 2020afe89df..7ffb2af5c5d 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -36,13 +36,7 @@ public: /** Initialize function node with function name. * Later during query analysis path function must be resolved. */ - explicit FunctionNode(String function_name_) - : function_name(function_name_) - { - children.resize(2); - children[parameters_child_index] = std::make_shared(); - children[arguments_child_index] = std::make_shared(); - } + explicit FunctionNode(String function_name_); /// Get name const String & getFunctionName() const diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index a69afa2eb4f..6643a254bab 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -30,6 +30,7 @@ const char * toString(QueryTreeNodeType type) case QueryTreeNodeType::FUNCTION: return "FUNCTION"; case QueryTreeNodeType::COLUMN: return "COLUMN"; case QueryTreeNodeType::LAMBDA: return "LAMBDA"; + case QueryTreeNodeType::SORT_COLUMN: return "SORT_COLUMN"; case QueryTreeNodeType::TABLE: return "TABLE"; case QueryTreeNodeType::TABLE_FUNCTION: return "TABLE_FUNCTION"; case QueryTreeNodeType::QUERY: return "QUERY"; diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 2343716e82b..38be6af5300 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -35,6 +35,7 @@ enum class QueryTreeNodeType FUNCTION, COLUMN, LAMBDA, + SORT_COLUMN, TABLE, TABLE_FUNCTION, QUERY, diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 93acfc98907..2b4f019f0d4 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -40,6 +40,7 @@ #include #include #include +#include #include #include #include @@ -88,6 +89,7 @@ namespace ErrorCodes extern const int INCORRECT_ELEMENT_OF_SET; extern const int TYPE_MISMATCH; extern const int AMBIGUOUS_IDENTIFIER; + extern const int INVALID_WITH_FILL_EXPRESSION; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -812,6 +814,8 @@ private: void resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); + void resolveSortColumnsNodeList(QueryTreeNodePtr & sort_columns_node_list, IdentifierResolveScope & scope); + void initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope); void initializeTableExpressionColumns(QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); @@ -2846,6 +2850,14 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes /// Lambda must be resolved by caller break; } + + case QueryTreeNodeType::SORT_COLUMN: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Sort column {} is not allowed in expression. In scope {}", + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } case QueryTreeNodeType::TABLE: { if (!allow_table_expression) @@ -2887,14 +2899,14 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes case QueryTreeNodeType::ARRAY_JOIN: { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Array join is not allowed {} in expression. In scope {}", + "Array join {} is not allowed in expression. In scope {}", node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } case QueryTreeNodeType::JOIN: { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Join is not allowed {} in expression. In scope {}", + "Join {} is not allowed in expression. In scope {}", node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } @@ -2958,6 +2970,58 @@ void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, Iden node_list = std::move(result_node_list); } +/** Resolve sort columns nodes list. + */ +void QueryAnalyzer::resolveSortColumnsNodeList(QueryTreeNodePtr & sort_columns_node_list, IdentifierResolveScope & scope) +{ + auto & sort_columns_node_list_typed = sort_columns_node_list->as(); + for (auto & node : sort_columns_node_list_typed.getNodes()) + { + auto & sort_column_node = node->as(); + resolveExpressionNode(sort_column_node.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + if (sort_column_node.hasFillFrom()) + { + resolveExpressionNode(sort_column_node.getFillFrom(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + const auto * constant_node = sort_column_node.getFillFrom()->as(); + if (!constant_node || !isColumnedAsNumber(constant_node->getResultType())) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL FROM expression must be constant with numeric type. Actual {}. In scope {}", + sort_column_node.getFillFrom()->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + if (sort_column_node.hasFillTo()) + { + resolveExpressionNode(sort_column_node.getFillTo(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + const auto * constant_node = sort_column_node.getFillTo()->as(); + if (!constant_node || !isColumnedAsNumber(constant_node->getResultType())) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL TO expression must be constant with numeric type. Actual {}. In scope {}", + sort_column_node.getFillFrom()->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + if (sort_column_node.hasFillStep()) + { + resolveExpressionNode(sort_column_node.getFillStep(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + const auto * constant_node = sort_column_node.getFillStep()->as(); + if (!constant_node) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL TO expression must be constant with numeric or interval type. Actual {}. In scope {}", + sort_column_node.getFillStep()->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + bool is_number = isColumnedAsNumber(constant_node->getResultType()); + bool is_interval = WhichDataType(constant_node->getResultType()).isInterval(); + if (!is_number && !is_interval) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL TO expression must be constant with numeric or interval type. Actual {}. In scope {}", + sort_column_node.getFillStep()->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + } +} + /** Initialize query join tree node. * * 1. Resolve identifiers. @@ -3447,7 +3511,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier QueryExpressionsAliasVisitor::Data data{scope}; QueryExpressionsAliasVisitor visitor(data); - if (!query_node_typed.getWith().getNodes().empty()) + if (query_node_typed.hasWith()) visitor.visit(query_node_typed.getWithNode()); if (!query_node_typed.getProjection().getNodes().empty()) @@ -3459,6 +3523,12 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.getWhere()) visitor.visit(query_node_typed.getWhere()); + if (query_node_typed.hasGroupBy()) + visitor.visit(query_node_typed.getGroupByNode()); + + if (query_node_typed.hasOrderBy()) + visitor.visit(query_node_typed.getOrderByNode()); + /// Register CTE subqueries and remove them from WITH section auto & with_nodes = query_node_typed.getWith().getNodes(); @@ -3511,7 +3581,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /// Resolve query node sections. - if (!query_node_typed.getWith().getNodes().empty()) + if (query_node_typed.hasWith()) resolveExpressionNodeList(query_node_typed.getWithNode(), scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); if (query_node_typed.getPrewhere()) @@ -3520,8 +3590,11 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.getWhere()) resolveExpressionNode(query_node_typed.getWhere(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - if (!query_node_typed.getGroupBy().getNodes().empty()) - resolveExpressionNodeList(query_node_typed.getGroupByNode(), scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); + if (query_node_typed.hasGroupBy()) + resolveExpressionNodeList(query_node_typed.getGroupByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + if (query_node_typed.hasOrderBy()) + resolveSortColumnsNodeList(query_node_typed.getOrderByNode(), scope); resolveExpressionNodeList(query_node_typed.getProjectionNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 35bee05b85e..e2d3e490ad7 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -27,6 +27,7 @@ QueryNode::QueryNode() children[with_child_index] = std::make_shared(); children[projection_child_index] = std::make_shared(); children[group_by_child_index] = std::make_shared(); + children[order_by_child_index] = std::make_shared(); } NamesAndTypesList QueryNode::computeProjectionColumns() const @@ -56,12 +57,13 @@ String QueryNode::getName() const { WriteBufferFromOwnString buffer; - if (!getWith().getNodes().empty()) + if (hasWith()) { buffer << getWith().getName(); + buffer << ' '; } - buffer << " SELECT "; + buffer << "SELECT "; buffer << getProjection().getName(); if (getJoinTree()) @@ -82,10 +84,8 @@ String QueryNode::getName() const buffer << getWhere()->getName(); } - if (!getGroupBy().getNodes().empty()) - { + if (hasGroupBy()) buffer << getGroupBy().getName(); - } return buffer.str(); } @@ -104,7 +104,7 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s if (!cte_name.empty()) buffer << ", cte_name: " << cte_name; - if (!getWith().getNodes().empty()) + if (hasWith()) { buffer << '\n' << std::string(indent + 2, ' ') << "WITH\n"; getWith().dumpTreeImpl(buffer, format_state, indent + 4); @@ -132,11 +132,17 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s getWhere()->dumpTreeImpl(buffer, format_state, indent + 4); } - if (!getGroupBy().getNodes().empty()) + if (hasGroupBy()) { buffer << '\n' << std::string(indent + 2, ' ') << "GROUP BY\n"; getGroupBy().dumpTreeImpl(buffer, format_state, indent + 4); } + + if (hasOrderBy()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "ORDER BY\n"; + getOrderBy().dumpTreeImpl(buffer, format_state, indent + 4); + } } bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const @@ -161,7 +167,7 @@ ASTPtr QueryNode::toASTImpl() const auto select_query = std::make_shared(); select_query->distinct = is_distinct; - if (!getWith().getNodes().empty()) + if (hasWith()) select_query->setExpression(ASTSelectQuery::Expression::WITH, getWith().toAST()); select_query->setExpression(ASTSelectQuery::Expression::SELECT, children[projection_child_index]->toAST()); @@ -176,17 +182,20 @@ ASTPtr QueryNode::toASTImpl() const if (getWhere()) select_query->setExpression(ASTSelectQuery::Expression::WHERE, getWhere()->toAST()); - if (!getGroupBy().getNodes().empty()) + if (hasGroupBy()) select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, getGroupBy().toAST()); + if (hasOrderBy()) + select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, getOrderBy().toAST()); + auto result_select_query = std::make_shared(); result_select_query->union_mode = SelectUnionMode::Unspecified; auto list_of_selects = std::make_shared(); list_of_selects->children.push_back(std::move(select_query)); - result_select_query->list_of_selects = std::move(list_of_selects); - result_select_query->children.push_back(std::move(select_query)); + result_select_query->children.push_back(std::move(list_of_selects)); + result_select_query->list_of_selects = result_select_query->children.back(); if (is_subquery) { diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 12684a8d059..e08c6a85287 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -57,6 +57,11 @@ public: return is_distinct; } + bool hasWith() const + { + return !getWith().getNodes().empty(); + } + const ListNode & getWith() const { return children[with_child_index]->as(); @@ -162,6 +167,31 @@ public: return children[group_by_child_index]; } + bool hasOrderBy() const + { + return !getOrderBy().getNodes().empty(); + } + + const ListNode & getOrderBy() const + { + return children[order_by_child_index]->as(); + } + + ListNode & getOrderBy() + { + return children[order_by_child_index]->as(); + } + + const QueryTreeNodePtr & getOrderByNode() const + { + return children[order_by_child_index]; + } + + QueryTreeNodePtr & getOrderByNode() + { + return children[order_by_child_index]; + } + /// Compute query node columns using projection section NamesAndTypesList computeProjectionColumns() const; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 45769d3029d..ee5da55859e 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -29,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -78,6 +80,8 @@ private: QueryTreeNodePtr buildSelectExpression(const ASTPtr & select_query, bool is_subquery, const std::string & cte_name) const; + QueryTreeNodePtr buildSortColumnList(const ASTPtr & order_by_expression_list) const; + QueryTreeNodePtr buildExpressionList(const ASTPtr & expression_list) const; QueryTreeNodePtr buildExpression(const ASTPtr & expression) const; @@ -215,9 +219,50 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q if (group_by_list) current_query_tree->getGroupByNode() = buildExpressionList(group_by_list); + auto select_order_by_list = select_query_typed.orderBy(); + if (select_order_by_list) + current_query_tree->getOrderByNode() = buildSortColumnList(select_order_by_list); + return current_query_tree; } +QueryTreeNodePtr QueryTreeBuilder::buildSortColumnList(const ASTPtr & order_by_expression_list) const +{ + auto list_node = std::make_shared(); + + auto & expression_list_typed = order_by_expression_list->as(); + list_node->getNodes().reserve(expression_list_typed.children.size()); + + for (auto & expression : expression_list_typed.children) + { + const auto & order_by_element = expression->as(); + + auto sort_direction = order_by_element.direction == 1 ? SortDirection::ASCENDING : SortDirection::DESCENDING; + std::optional nulls_sort_direction; + if (order_by_element.nulls_direction_was_explicitly_specified) + nulls_sort_direction = order_by_element.nulls_direction == 1 ? SortDirection::ASCENDING : SortDirection::DESCENDING; + + std::shared_ptr collator; + if (order_by_element.collation) + collator = std::make_shared(order_by_element.collation->as().value.get()); + + const auto & sort_expression_ast = order_by_element.children.at(0); + auto sort_expression = buildExpression(sort_expression_ast); + auto sort_column_node = std::make_shared(std::move(sort_expression), sort_direction, nulls_sort_direction, std::move(collator)); + + if (order_by_element.fill_from) + sort_column_node->getFillFrom() = buildExpression(order_by_element.fill_from); + if (order_by_element.fill_to) + sort_column_node->getFillTo() = buildExpression(order_by_element.fill_to); + if (order_by_element.fill_step) + sort_column_node->getFillStep() = buildExpression(order_by_element.fill_step); + + list_node->getNodes().push_back(std::move(sort_column_node)); + } + + return list_node; +} + QueryTreeNodePtr QueryTreeBuilder::buildExpressionList(const ASTPtr & expression_list) const { auto list_node = std::make_shared(); @@ -456,7 +501,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select for (const auto & argument : function_arguments_list) { if (argument->as() || argument->as() || argument->as()) - node->getArguments().getNodes().push_back(buildSelectOrUnionExpression(argument, true /*is_subquery*/, {} /*cte_name*/)); + node->getArguments().getNodes().push_back(buildSelectOrUnionExpression(argument, false /*is_subquery*/, {} /*cte_name*/)); else node->getArguments().getNodes().push_back(buildExpression(argument)); } diff --git a/src/Analyzer/SortColumnNode.cpp b/src/Analyzer/SortColumnNode.cpp new file mode 100644 index 00000000000..c55946db4f8 --- /dev/null +++ b/src/Analyzer/SortColumnNode.cpp @@ -0,0 +1,158 @@ +#include + +#include + +#include +#include + +#include +#include +#include + +namespace DB +{ + +const char * toString(SortDirection sort_direction) +{ + switch (sort_direction) + { + case SortDirection::ASCENDING: return "ASCENDING"; + case SortDirection::DESCENDING: return "DESCENDING"; + } +} + +SortColumnNode::SortColumnNode(QueryTreeNodePtr expression_, + SortDirection sort_direction_, + std::optional nulls_sort_direction_, + std::shared_ptr collator_) + : sort_direction(sort_direction_) + , nulls_sort_direction(nulls_sort_direction_) + , collator(std::move(collator_)) +{ + children.resize(children_size); + children[sort_expression_child_index] = std::move(expression_); +} + +String SortColumnNode::getName() const +{ + String result = getExpression()->getName(); + + if (sort_direction == SortDirection::ASCENDING) + result += " ASC"; + else + result += " DESC"; + + if (nulls_sort_direction) + { + if (*nulls_sort_direction == SortDirection::ASCENDING) + result += " NULLS FIRST"; + else + result += " NULLS LAST"; + } + + if (hasWithFill()) + result += " WITH FILL"; + + if (hasFillFrom()) + result += " FROM " + getFillFrom()->getName(); + + if (hasFillStep()) + result += " STEP " + getFillStep()->getName(); + + if (hasFillTo()) + result += " TO " + getFillTo()->getName(); + + return result; +} + +void SortColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const +{ + buffer << std::string(indent, ' ') << "SORT_COLUMN id: " << format_state.getNodeId(this); + + buffer << ", sort_direction: " << toString(sort_direction); + if (nulls_sort_direction) + buffer << ", nulls_sort_direction: " << toString(*nulls_sort_direction); + + if (collator) + buffer << ", collator: " << collator->getLocale(); + + buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION\n"; + getExpression()->dumpTreeImpl(buffer, format_state, indent + 4); + + if (hasFillFrom()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "FILL FROM\n"; + getFillFrom()->dumpTreeImpl(buffer, format_state, indent + 4); + } + + if (hasFillTo()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "FILL TO\n"; + getFillTo()->dumpTreeImpl(buffer, format_state, indent + 4); + } + + if (hasFillStep()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "FILL STEP\n"; + getFillStep()->dumpTreeImpl(buffer, format_state, indent + 4); + } +} + +bool SortColumnNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + if (sort_direction != rhs_typed.sort_direction || + nulls_sort_direction != rhs_typed.nulls_sort_direction) + return false; + + if (!collator && !rhs_typed.collator) + return true; + else if (collator && !rhs_typed.collator) + return false; + else if (!collator && rhs_typed.collator) + return false; + + return collator->getLocale() == rhs_typed.collator->getLocale(); +} + +void SortColumnNode::updateTreeHashImpl(HashState & hash_state) const +{ + hash_state.update(sort_direction); + hash_state.update(nulls_sort_direction); + + if (collator) + { + const auto & locale = collator->getLocale(); + + hash_state.update(locale.size()); + hash_state.update(locale); + } +} + +ASTPtr SortColumnNode::toASTImpl() const +{ + auto result = std::make_shared(); + result->direction = sort_direction == SortDirection::ASCENDING ? 1 : -1; + result->nulls_direction = result->direction; + if (nulls_sort_direction) + result->nulls_direction = *nulls_sort_direction == SortDirection::ASCENDING ? 1 : -1; + + result->nulls_direction_was_explicitly_specified = nulls_sort_direction.has_value(); + if (collator) + result->collation = std::make_shared(Field(collator->getLocale())); + + result->with_fill = hasWithFill(); + result->fill_from = hasFillFrom() ? getFillFrom()->toAST() : nullptr; + result->fill_to = hasFillTo() ? getFillTo()->toAST() : nullptr; + result->fill_step = hasFillStep() ? getFillStep()->toAST() : nullptr; + result->children.push_back(getExpression()->toAST()); + + return result; +} + +QueryTreeNodePtr SortColumnNode::cloneImpl() const +{ + return std::make_shared(nullptr, sort_direction, nulls_sort_direction, collator); +} + +} diff --git a/src/Analyzer/SortColumnNode.h b/src/Analyzer/SortColumnNode.h new file mode 100644 index 00000000000..04642a346a2 --- /dev/null +++ b/src/Analyzer/SortColumnNode.h @@ -0,0 +1,152 @@ +#pragma once + +#include + +#include +#include + +namespace DB +{ + +/** Sort column node represents sort column descripion that is part of ORDER BY in query tree. + * Example: SELECT * FROM test_table ORDER BY sort_column_1, sort_column_2; + * Sort column optionally contain collation, fill from, fill to, and fill step. + */ +class SortColumnNode; +using SortColumnNodePtr = std::shared_ptr; + +enum class SortDirection +{ + ASCENDING = 0, + DESCENDING = 1 +}; + +const char * toString(SortDirection sort_direction); + +class SortColumnNode final : public IQueryTreeNode +{ +public: + /// Initialize sort column node with sort expression + explicit SortColumnNode(QueryTreeNodePtr expression_, + SortDirection sort_direction_ = SortDirection::ASCENDING, + std::optional nulls_sort_direction_ = {}, + std::shared_ptr collator_ = nullptr); + + /// Get sort expression + const QueryTreeNodePtr & getExpression() const + { + return children[sort_expression_child_index]; + } + + /// Get sort expression + QueryTreeNodePtr & getExpression() + { + return children[sort_expression_child_index]; + } + + /// Has with fill + bool hasWithFill() const + { + return hasFillFrom() || hasFillStep() || hasFillTo(); + } + + /// Has fill from + bool hasFillFrom() const + { + return children[fill_from_child_index] != nullptr; + } + + /// Get fill from + const QueryTreeNodePtr & getFillFrom() const + { + return children[fill_from_child_index]; + } + + /// Get fill from + QueryTreeNodePtr & getFillFrom() + { + return children[fill_from_child_index]; + } + + /// Has fill to + bool hasFillTo() const + { + return children[fill_to_child_index] != nullptr; + } + + /// Get fill to + const QueryTreeNodePtr & getFillTo() const + { + return children[fill_to_child_index]; + } + + /// Get fill to + QueryTreeNodePtr & getFillTo() + { + return children[fill_to_child_index]; + } + + bool hasFillStep() const + { + return children[fill_step_child_index] != nullptr; + } + + /// Get fill step + const QueryTreeNodePtr & getFillStep() const + { + return children[fill_step_child_index]; + } + + /// Get fill to + QueryTreeNodePtr & getFillStep() + { + return children[fill_step_child_index]; + } + + /// Get collator + const std::shared_ptr & getCollator() const + { + return collator; + } + + SortDirection getSortDirection() const + { + return sort_direction; + } + + std::optional getNullsSortDirection() const + { + return nulls_sort_direction; + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::SORT_COLUMN; + } + + String getName() const override; + + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + + void updateTreeHashImpl(HashState & hash_state) const override; + +protected: + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + static constexpr size_t sort_expression_child_index = 0; + static constexpr size_t fill_from_child_index = 1; + static constexpr size_t fill_to_child_index = 2; + static constexpr size_t fill_step_child_index = 3; + static constexpr size_t children_size = fill_step_child_index + 1; + + SortDirection sort_direction = SortDirection::ASCENDING; + std::optional nulls_sort_direction; + std::shared_ptr collator; +}; + +} diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 3ddd45ad67c..d764bcba0ef 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -21,6 +21,8 @@ #include #include #include +#include +#include #include #include @@ -37,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -51,6 +54,7 @@ #include #include #include +#include #include namespace DB @@ -65,6 +69,7 @@ namespace ErrorCodes extern const int INVALID_JOIN_ON_EXPRESSION; extern const int ILLEGAL_AGGREGATION; extern const int NOT_AN_AGGREGATE; + extern const int INVALID_WITH_FILL_EXPRESSION; } /** ClickHouse query planner. @@ -88,6 +93,9 @@ namespace ErrorCodes * TODO: Support max streams * TODO: Support GROUPINS SETS, const aggregation keys, overflow row * TODO: Simplify buildings SETS for IN function + * TODO: Support interpolate, LIMIT BY. + * TODO: Support ORDER BY read in order optimization + * TODO: Support GROUP BY read in order optimization */ namespace @@ -1102,7 +1110,6 @@ void Planner::buildQueryPlanIfNeeded() ValidateGroupByColumnsVisitor::Data validate_group_by_visitor_data(aggregate_keys_set, *planner_context); ValidateGroupByColumnsVisitor validate_columns_visitor(validate_group_by_visitor_data); - validate_columns_visitor.visit(query_node.getProjectionNode()); auto aggregate_step = std::make_unique(std::move(group_by_actions), ActionsChainStep::AvailableOutputColumnsStrategy::OUTPUT_NODES, aggregates_columns); @@ -1110,6 +1117,34 @@ void Planner::buildQueryPlanIfNeeded() aggregate_step_index = actions_chain.getLastStepIndex(); } + std::optional before_order_by_step_index; + if (query_node.hasOrderBy()) + { + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & order_by_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + ActionsDAGPtr actions_dag = std::make_shared(order_by_input); + auto & actions_dag_outputs = actions_dag->getOutputs(); + actions_dag_outputs.clear(); + + PlannerActionsVisitor actions_visitor(planner_context); + + /** We add only sort column sort expression in before ORDER BY actions DAG. + * WITH fill expressions must be constant nodes. + */ + auto & order_by_node_list = query_node.getOrderByNode()->as(); + for (auto & sort_column_node : order_by_node_list.getNodes()) + { + auto & sort_column_node_typed = sort_column_node->as(); + auto expression_dag_index_nodes = actions_visitor.visit(actions_dag, sort_column_node_typed.getExpression()); + actions_dag_outputs.insert(actions_dag_outputs.end(), expression_dag_index_nodes.begin(), expression_dag_index_nodes.end()); + } + + auto actions_step_before_order_by = std::make_unique(std::move(actions_dag)); + actions_chain.addStep(std::move(actions_step_before_order_by)); + before_order_by_step_index = actions_chain.getLastStepIndex(); + } + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); auto projection_actions = convertExpressionNodeIntoDAG(query_node.getProjectionNode(), projection_input, planner_context); @@ -1152,13 +1187,13 @@ void Planner::buildQueryPlanIfNeeded() actions_chain.addStep(std::make_unique(std::move(projection_actions))); size_t projection_action_step_index = actions_chain.getLastStepIndex(); - // std::cout << "Chain dump before finalize" << std::endl; - // std::cout << actions_chain.dump() << std::endl; + std::cout << "Chain dump before finalize" << std::endl; + std::cout << actions_chain.dump() << std::endl; actions_chain.finalize(); - // std::cout << "Chain dump after finalize" << std::endl; - // std::cout << actions_chain.dump() << std::endl; + std::cout << "Chain dump after finalize" << std::endl; + std::cout << actions_chain.dump() << std::endl; if (where_action_step_index) { @@ -1283,6 +1318,55 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(distinct_step)); } + if (before_order_by_step_index) + { + auto & aggregate_actions_chain_node = actions_chain.at(*before_order_by_step_index); + auto expression_step_before_order_by = std::make_unique(query_plan.getCurrentDataStream(), + aggregate_actions_chain_node->getActions()); + expression_step_before_order_by->setStepDescription("Before ORDER BY"); + query_plan.addStep(std::move(expression_step_before_order_by)); + } + + if (query_node.hasOrderBy()) + { + SortDescription sort_description = extractSortDescription(query_node.getOrderByNode(), *planner_context); + String sort_description_dump = dumpSortDescription(sort_description); + + UInt64 limit = 0; + + const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); + + /// Merge the sorted blocks. + auto sorting_step = std::make_unique( + query_plan.getCurrentDataStream(), + sort_description, + settings.max_block_size, + limit, + SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), + settings.max_bytes_before_remerge_sort, + settings.remerge_sort_lowered_memory_bytes_ratio, + settings.max_bytes_before_external_sort, + planner_context->getQueryContext()->getTemporaryVolume(), + settings.min_free_disk_space_for_temporary_data); + + sorting_step->setStepDescription("Sorting for ORDER BY"); + query_plan.addStep(std::move(sorting_step)); + + SortDescription fill_description; + for (auto & description : sort_description) + { + if (description.with_fill) + fill_description.push_back(description); + } + + if (!fill_description.empty()) + { + InterpolateDescriptionPtr interpolate_descr = nullptr; + auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_description), interpolate_descr); + query_plan.addStep(std::move(filling_step)); + } + } + auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), actions_chain[projection_action_step_index]->getActions()); projection_step->setStepDescription("Projection"); query_plan.addStep(std::move(projection_step)); diff --git a/src/Planner/PlannerSorting.cpp b/src/Planner/PlannerSorting.cpp new file mode 100644 index 00000000000..869d1611500 --- /dev/null +++ b/src/Planner/PlannerSorting.cpp @@ -0,0 +1,155 @@ +#include + +#include + +#include + +#include + +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INVALID_WITH_FILL_EXPRESSION; +} + +namespace +{ + +std::pair extractWithFillValue(const QueryTreeNodePtr & node) +{ + const auto & constant_node = node->as(); + + std::pair result; + result.first = constant_node.getConstantValue(); + result.second = constant_node.getResultType(); + + if (!isColumnedAsNumber(result.second)) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL expression must be constant with numeric type"); + + return result; +} + +std::pair> extractWithFillStepValue(const QueryTreeNodePtr & node) +{ + const auto & constant_node = node->as(); + const auto & constant_node_result_type = constant_node.getResultType(); + if (const auto * type_interval = typeid_cast(constant_node_result_type.get())) + return std::make_pair(constant_node.getConstantValue(), type_interval->getKind()); + + if (!isColumnedAsNumber(constant_node_result_type)) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL expression must be constant with numeric type"); + + return {constant_node.getConstantValue(), {}}; +} + +FillColumnDescription extractWithFillDescription(const SortColumnNode & sort_column_node) +{ + FillColumnDescription fill_column_description; + + if (sort_column_node.hasFillFrom()) + { + auto extract_result = extractWithFillValue(sort_column_node.getFillFrom()); + fill_column_description.fill_from = std::move(extract_result.first); + fill_column_description.fill_from_type = std::move(extract_result.second); + } + + if (sort_column_node.hasFillTo()) + { + auto extract_result = extractWithFillValue(sort_column_node.getFillTo()); + fill_column_description.fill_to = std::move(extract_result.first); + fill_column_description.fill_to_type = std::move(extract_result.second); + } + + if (sort_column_node.hasFillStep()) + { + auto extract_result = extractWithFillStepValue(sort_column_node.getFillStep()); + fill_column_description.fill_step = std::move(extract_result.first); + fill_column_description.step_kind = std::move(extract_result.second); + } + else + { + fill_column_description.fill_step = Field(sort_column_node.getSortDirection() == SortDirection::ASCENDING ? 1 : -1); + } + + if (applyVisitor(FieldVisitorAccurateEquals(), fill_column_description.fill_step, Field{0})) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL STEP value cannot be zero"); + + if (sort_column_node.getSortDirection() == SortDirection::ASCENDING) + { + if (applyVisitor(FieldVisitorAccurateLess(), fill_column_description.fill_step, Field{0})) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL STEP value cannot be negative for sorting in ascending direction"); + + if (!fill_column_description.fill_from.isNull() && !fill_column_description.fill_to.isNull() && + applyVisitor(FieldVisitorAccurateLess(), fill_column_description.fill_to, fill_column_description.fill_from)) + { + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL TO value cannot be less than FROM value for sorting in ascending direction"); + } + } + else + { + if (applyVisitor(FieldVisitorAccurateLess(), Field{0}, fill_column_description.fill_step)) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL STEP value cannot be positive for sorting in descending direction"); + + if (!fill_column_description.fill_from.isNull() && !fill_column_description.fill_to.isNull() && + applyVisitor(FieldVisitorAccurateLess(), fill_column_description.fill_from, fill_column_description.fill_to)) + { + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL FROM value cannot be less than TO value for sorting in descending direction"); + } + } + + return fill_column_description; +} + +} + +SortDescription extractSortDescription(const QueryTreeNodePtr & order_by_node, const PlannerContext & planner_context) +{ + auto & order_by_list_node = order_by_node->as(); + + SortDescription sort_column_description; + sort_column_description.reserve(order_by_list_node.getNodes().size()); + + for (const auto & sort_column_node : order_by_list_node.getNodes()) + { + auto & sort_column_node_typed = sort_column_node->as(); + + auto column_name = calculateActionsDAGNodeName(sort_column_node_typed.getExpression().get(), planner_context); + std::shared_ptr collator = sort_column_node_typed.getCollator(); + int direction = sort_column_node_typed.getSortDirection() == SortDirection::ASCENDING ? 1 : -1; + int nulls_direction = direction; + + auto nulls_sort_direction = sort_column_node_typed.getNullsSortDirection(); + if (nulls_sort_direction) + nulls_direction = *nulls_sort_direction == SortDirection::ASCENDING ? 1 : -1; + + if (sort_column_node_typed.hasWithFill()) + { + FillColumnDescription fill_description = extractWithFillDescription(sort_column_node_typed); + sort_column_description.emplace_back(column_name, direction, nulls_direction, collator, true /*with_fill*/, fill_description); + } + else + { + sort_column_description.emplace_back(column_name, direction, nulls_direction, collator); + } + } + + const auto & settings = planner_context.getQueryContext()->getSettingsRef(); + sort_column_description.compile_sort_description = settings.compile_sort_description; + sort_column_description.min_count_to_compile_sort_description = settings.min_count_to_compile_sort_description; + + return sort_column_description; +} + +} diff --git a/src/Planner/PlannerSorting.h b/src/Planner/PlannerSorting.h new file mode 100644 index 00000000000..ae3d2d1acfe --- /dev/null +++ b/src/Planner/PlannerSorting.h @@ -0,0 +1,14 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +/// Extract sort description from query order by node +SortDescription extractSortDescription(const QueryTreeNodePtr & order_by_node, const PlannerContext & planner_context); + +} + diff --git a/src/TableFunctions/TableFunctionViewIfPermitted.cpp b/src/TableFunctions/TableFunctionViewIfPermitted.cpp index 176db0915d4..ba3d2cb9d16 100644 --- a/src/TableFunctions/TableFunctionViewIfPermitted.cpp +++ b/src/TableFunctions/TableFunctionViewIfPermitted.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include @@ -39,14 +38,7 @@ void TableFunctionViewIfPermitted::parseArguments(const ASTPtr & ast_function, C getName()); const auto & arguments = function->arguments->children; - auto select_argument = arguments[0]; - auto * subquery = arguments[0]->as(); - - if (subquery) - select_argument = subquery->children[0]; - - auto * select = select_argument->as(); - + auto * select = arguments[0]->as(); if (!select) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}' requires a SELECT query as its first argument", getName()); create.set(create.select, select->clone()); From ce9ea7ca0bcc7cc5dbb6949c0ea92c4122d7f276 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Aug 2022 15:31:03 +0200 Subject: [PATCH 077/188] Fixed tests --- src/Planner/PlannerActionsVisitor.cpp | 2 ++ ....reference => 02376_analyzer_in_function_subquery.reference} | 0 ...ion_subquery.sql => 02376_analyzer_in_function_subquery.sql} | 0 3 files changed, 2 insertions(+) rename tests/queries/0_stateless/{02376_function_in_function_subquery.reference => 02376_analyzer_in_function_subquery.reference} (100%) rename tests/queries/0_stateless/{02376_function_in_function_subquery.sql => 02376_analyzer_in_function_subquery.sql} (100%) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index b320fed2803..7d001aa14cd 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -575,6 +575,8 @@ String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerCon result = buffer.str(); break; } + case QueryTreeNodeType::UNION: + [[fallthrough]]; case QueryTreeNodeType::QUERY: { auto query_hash = node->getTreeHash(); diff --git a/tests/queries/0_stateless/02376_function_in_function_subquery.reference b/tests/queries/0_stateless/02376_analyzer_in_function_subquery.reference similarity index 100% rename from tests/queries/0_stateless/02376_function_in_function_subquery.reference rename to tests/queries/0_stateless/02376_analyzer_in_function_subquery.reference diff --git a/tests/queries/0_stateless/02376_function_in_function_subquery.sql b/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql similarity index 100% rename from tests/queries/0_stateless/02376_function_in_function_subquery.sql rename to tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql From 1cbf96d1652127d2c83ede275043a2f30824155b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Aug 2022 18:23:26 +0200 Subject: [PATCH 078/188] Added LIMIT, OFFSET support --- src/Analyzer/FunctionNode.cpp | 7 +-- src/Analyzer/QueryAnalysisPass.cpp | 66 +++++++++++++++++++++++---- src/Analyzer/QueryNode.cpp | 28 +++++++++++- src/Analyzer/QueryNode.h | 44 +++++++++++++++++- src/Analyzer/QueryTreeBuilder.cpp | 9 ++++ src/Planner/Planner.cpp | 61 +++++++++++++++++++++++-- src/Planner/PlannerActionsVisitor.cpp | 4 +- 7 files changed, 195 insertions(+), 24 deletions(-) diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index 06f0f248847..518bb9709d6 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -133,11 +133,8 @@ ASTPtr FunctionNode::toASTImpl() const } const auto & arguments = getArguments(); - if (!arguments.getNodes().empty()) - { - function_ast->children.push_back(arguments.toAST()); - function_ast->arguments = function_ast->children.back(); - } + function_ast->children.push_back(arguments.toAST()); + function_ast->arguments = function_ast->children.back(); return function_ast; } diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 2b4f019f0d4..763760f7c52 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -90,6 +90,7 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; extern const int AMBIGUOUS_IDENTIFIER; extern const int INVALID_WITH_FILL_EXPRESSION; + extern const int INVALID_LIMIT_EXPRESSION; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -363,20 +364,30 @@ struct TableExpressionColumns std::unordered_map column_name_to_column_node; std::unordered_set column_identifier_first_parts; - bool canBindIdentifier(IdentifierView identifier) + bool hasFullIdentifierName(IdentifierView identifier) const { - return column_identifier_first_parts.find(std::string(identifier.at(0))) != column_identifier_first_parts.end(); + return column_name_to_column_node.contains(std::string(identifier.getFullName())); } - [[maybe_unused]] void dump(WriteBuffer & buffer) + bool canBindIdentifier(IdentifierView identifier) const + { + return column_identifier_first_parts.contains(std::string(identifier.at(0))); + } + + [[maybe_unused]] void dump(WriteBuffer & buffer) const { buffer << "Columns size " << column_name_to_column_node.size() << '\n'; - for (auto & [column_name, column_node] : column_name_to_column_node) - { - buffer << "Column name " << column_name << " column node " << column_node->formatASTForErrorMessage(); - buffer << " is alias " << column_node->hasExpression() << '\n'; - } + for (const auto & [column_name, column_node] : column_name_to_column_node) + buffer << "Column name " << column_name << " column node " << column_node->dumpTree() << '\n'; + } + + [[maybe_unused]] String dump() const + { + WriteBufferFromOwnString buffer; + dump(buffer); + + return buffer.str(); } }; @@ -1380,13 +1391,14 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo return result_expression; }; - /** If identifier first part binds to some column start. Then we can try to find whole identifier in table. + /** If identifier first part binds to some column start or table has full identifier name. Then we can try to find whole identifier in table. * 1. Try to bind identifier first part to column in table, if true get full identifier from table or throw exception. * 2. Try to bind identifier first part to table name or storage alias, if true remove first part and try to get full identifier from table or throw exception. * Storage alias works for subquery, table function as well. * 3. Try to bind identifier first parts to database name and table name, if true remove first two parts and try to get full identifier from table or throw exception. */ - if (storage_columns.canBindIdentifier(IdentifierView(identifier))) + if (storage_columns.canBindIdentifier(IdentifierView(identifier)) || + storage_columns.hasFullIdentifierName(IdentifierView(identifier))) return resolve_identifier_from_storage_or_throw(0 /*identifier_column_qualifier_parts*/); if (identifier.getPartsSize() == 1) @@ -3596,6 +3608,40 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasOrderBy()) resolveSortColumnsNodeList(query_node_typed.getOrderByNode(), scope); + if (query_node_typed.hasLimit()) + { + resolveExpressionNode(query_node_typed.getLimit(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + const auto * limit_constant_node = query_node_typed.getLimit()->as(); + if (!limit_constant_node || !isNativeNumber(limit_constant_node->getResultType())) + throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, + "Limit expression must be constant with numeric type. Actual {}. In scope {}", + query_node_typed.getLimit()->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + Field converted = convertFieldToType(limit_constant_node->getConstantValue(), DataTypeUInt64()); + if (converted.isNull()) + throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, + "Limit numeric constant expression is not representable as UInt64"); + } + + if (query_node_typed.getOffset()) + { + resolveExpressionNode(query_node_typed.getOffset(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + const auto * offset_constant_node = query_node_typed.getOffset()->as(); + if (!offset_constant_node || !isNativeNumber(offset_constant_node->getResultType())) + throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, + "Offset expression must be constant with numeric type. Actual {}. In scope {}", + query_node_typed.getLimit()->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + Field converted = convertFieldToType(offset_constant_node->getConstantValue(), DataTypeUInt64()); + if (converted.isNull()) + throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, + "Offset numeric constant expression is not representable as UInt64"); + } + resolveExpressionNodeList(query_node_typed.getProjectionNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); /** WITH section can be safely removed, because WITH section only can provide aliases to query expressions diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index e2d3e490ad7..5253bb15333 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -100,6 +100,7 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << ", is_subquery: " << is_subquery; buffer << ", is_cte: " << is_cte; buffer << ", is_distinct: " << is_distinct; + buffer << ", is_limit_with_ties: " << is_limit_with_ties; if (!cte_name.empty()) buffer << ", cte_name: " << cte_name; @@ -143,12 +144,28 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << '\n' << std::string(indent + 2, ' ') << "ORDER BY\n"; getOrderBy().dumpTreeImpl(buffer, format_state, indent + 4); } + + if (hasLimit()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "LIMIT\n"; + getLimit()->dumpTreeImpl(buffer, format_state, indent + 4); + } + + if (hasOffset()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "OFFSET\n"; + getOffset()->dumpTreeImpl(buffer, format_state, indent + 4); + } } bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); - return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name && is_distinct == rhs_typed.is_distinct; + return is_subquery == rhs_typed.is_subquery && + is_cte == rhs_typed.is_cte && + cte_name == rhs_typed.cte_name && + is_distinct == rhs_typed.is_distinct && + is_limit_with_ties == rhs_typed.is_limit_with_ties; } void QueryNode::updateTreeHashImpl(HashState & state) const @@ -160,6 +177,7 @@ void QueryNode::updateTreeHashImpl(HashState & state) const state.update(cte_name); state.update(is_distinct); + state.update(is_limit_with_ties); } ASTPtr QueryNode::toASTImpl() const @@ -188,6 +206,12 @@ ASTPtr QueryNode::toASTImpl() const if (hasOrderBy()) select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, getOrderBy().toAST()); + if (hasLimit()) + select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, getLimit()->toAST()); + + if (hasOffset()) + select_query->setExpression(ASTSelectQuery::Expression::LIMIT_OFFSET, getOffset()->toAST()); + auto result_select_query = std::make_shared(); result_select_query->union_mode = SelectUnionMode::Unspecified; @@ -216,6 +240,8 @@ QueryTreeNodePtr QueryNode::cloneImpl() const result_query_node->is_subquery = is_subquery; result_query_node->is_cte = is_cte; + result_query_node->is_distinct = is_distinct; + result_query_node->is_limit_with_ties = is_limit_with_ties; result_query_node->cte_name = cte_name; return result_query_node; diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index e08c6a85287..8405517c246 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -57,6 +57,16 @@ public: return is_distinct; } + void setIsLimitWithTies(bool is_limit_with_ties_value) + { + is_limit_with_ties = is_limit_with_ties_value; + } + + bool isLimitWithTies() const + { + return is_limit_with_ties; + } + bool hasWith() const { return !getWith().getNodes().empty(); @@ -192,6 +202,36 @@ public: return children[order_by_child_index]; } + bool hasLimit() const + { + return children[limit_child_index] != nullptr; + } + + const QueryTreeNodePtr & getLimit() const + { + return children[limit_child_index]; + } + + QueryTreeNodePtr & getLimit() + { + return children[limit_child_index]; + } + + bool hasOffset() const + { + return children[offset_child_index] != nullptr; + } + + const QueryTreeNodePtr & getOffset() const + { + return children[offset_child_index]; + } + + QueryTreeNodePtr & getOffset() + { + return children[offset_child_index]; + } + /// Compute query node columns using projection section NamesAndTypesList computeProjectionColumns() const; @@ -217,6 +257,7 @@ private: bool is_subquery = false; bool is_cte = false; bool is_distinct = false; + bool is_limit_with_ties = false; std::string cte_name; static constexpr size_t with_child_index = 0; @@ -228,7 +269,8 @@ private: static constexpr size_t having_child_index = 6; static constexpr size_t order_by_child_index = 7; static constexpr size_t limit_child_index = 8; - static constexpr size_t children_size = limit_child_index + 1; + static constexpr size_t offset_child_index = 9; + static constexpr size_t children_size = offset_child_index + 1; }; } diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index ee5da55859e..9799d6491dd 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -195,6 +195,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q current_query_tree->setIsCTE(!cte_name.empty()); current_query_tree->setCTEName(cte_name); current_query_tree->setIsDistinct(select_query_typed.distinct); + current_query_tree->setIsLimitWithTies(select_query_typed.limit_with_ties); current_query_tree->getJoinTree() = buildJoinTree(select_query_typed.tables()); current_query_tree->setOriginalAST(select_query); @@ -223,6 +224,14 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q if (select_order_by_list) current_query_tree->getOrderByNode() = buildSortColumnList(select_order_by_list); + auto select_limit = select_query_typed.limitLength(); + if (select_limit) + current_query_tree->getLimit() = buildExpression(select_limit); + + auto select_offset = select_query_typed.limitOffset(); + if (select_offset) + current_query_tree->getOffset() = buildExpression(select_offset); + return current_query_tree; } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d764bcba0ef..27bf77210ec 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -23,6 +23,8 @@ #include #include #include +#include +#include #include #include @@ -263,6 +265,7 @@ public: const NameSet & group_by_keys_column_names; const PlannerContext & planner_context; + QueryTreeNodeToName node_to_name; }; static void visit(const QueryTreeNodePtr & node, Data & data) @@ -1111,6 +1114,7 @@ void Planner::buildQueryPlanIfNeeded() ValidateGroupByColumnsVisitor::Data validate_group_by_visitor_data(aggregate_keys_set, *planner_context); ValidateGroupByColumnsVisitor validate_columns_visitor(validate_group_by_visitor_data); validate_columns_visitor.visit(query_node.getProjectionNode()); + validate_columns_visitor.visit(query_node.getOrderByNode()); auto aggregate_step = std::make_unique(std::move(group_by_actions), ActionsChainStep::AvailableOutputColumnsStrategy::OUTPUT_NODES, aggregates_columns); actions_chain.addStep(std::move(aggregate_step)); @@ -1187,13 +1191,13 @@ void Planner::buildQueryPlanIfNeeded() actions_chain.addStep(std::make_unique(std::move(projection_actions))); size_t projection_action_step_index = actions_chain.getLastStepIndex(); - std::cout << "Chain dump before finalize" << std::endl; - std::cout << actions_chain.dump() << std::endl; + // std::cout << "Chain dump before finalize" << std::endl; + // std::cout << actions_chain.dump() << std::endl; actions_chain.finalize(); - std::cout << "Chain dump after finalize" << std::endl; - std::cout << actions_chain.dump() << std::endl; + // std::cout << "Chain dump after finalize" << std::endl; + // std::cout << actions_chain.dump() << std::endl; if (where_action_step_index) { @@ -1327,9 +1331,11 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(expression_step_before_order_by)); } + SortDescription sort_description; + if (query_node.hasOrderBy()) { - SortDescription sort_description = extractSortDescription(query_node.getOrderByNode(), *planner_context); + sort_description = extractSortDescription(query_node.getOrderByNode(), *planner_context); String sort_description_dump = dumpSortDescription(sort_description); UInt64 limit = 0; @@ -1367,6 +1373,51 @@ void Planner::buildQueryPlanIfNeeded() } } + UInt64 limit_offset = 0; + if (query_node.hasOffset()) + { + /// Validated during query analysis stage + limit_offset = query_node.getOffset()->as().getConstantValue().safeGet(); + } + + if (query_node.hasLimit()) + { + const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); + bool always_read_till_end = settings.exact_rows_before_limit; + bool limit_with_ties = query_node.isLimitWithTies(); + + /// Validated during query analysis stage + UInt64 limit_length = query_node.getLimit()->as().getConstantValue().safeGet(); + + SortDescription limit_with_ties_sort_description; + + if (query_node.isLimitWithTies()) + { + /// Validated during parser stage + if (!query_node.hasOrderBy()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "LIMIT WITH TIES without ORDER BY"); + + limit_with_ties_sort_description = sort_description; + } + + auto limit = std::make_unique(query_plan.getCurrentDataStream(), + limit_length, + limit_offset, + always_read_till_end, + limit_with_ties, + limit_with_ties_sort_description); + + if (limit_with_ties) + limit->setStepDescription("LIMIT WITH TIES"); + + query_plan.addStep(std::move(limit)); + } + else if (query_node.hasOffset()) + { + auto offsets_step = std::make_unique(query_plan.getCurrentDataStream(), limit_offset); + query_plan.addStep(std::move(offsets_step)); + } + auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), actions_chain[projection_action_step_index]->getActions()); projection_step->setStepDescription("Projection"); query_plan.addStep(std::move(projection_step)); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 7d001aa14cd..0641a182ca8 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -526,8 +526,8 @@ String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerCon } case QueryTreeNodeType::CONSTANT: { - std::string constant_value_dump = node->as().getConstantValue().dump(); - result = "__constant_" + constant_value_dump; + const auto & constant_node = node->as(); + result = "__constant_" + constant_node.getName() + "_" + constant_node.getResultType()->getName(); break; } case QueryTreeNodeType::FUNCTION: From de1f470cfe7f19a6671b52ddf785d0a82683656e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 27 Aug 2022 15:16:02 +0200 Subject: [PATCH 079/188] Planner simplified creation of sets for IN function --- src/Planner/Planner.cpp | 4 +- src/Planner/PlannerActionsVisitor.cpp | 45 +------------ src/Planner/PlannerActionsVisitor.h | 7 +- src/Planner/PlannerCollectSets.cpp | 94 +++++++++++++++++++++++++++ src/Planner/PlannerCollectSets.h | 15 +++++ src/Planner/PlannerContext.cpp | 46 +++++++++++++ src/Planner/PlannerContext.h | 44 ++++++------- 7 files changed, 185 insertions(+), 70 deletions(-) create mode 100644 src/Planner/PlannerCollectSets.cpp create mode 100644 src/Planner/PlannerCollectSets.h diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 27bf77210ec..bb1a1c2b145 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -58,6 +58,7 @@ #include #include #include +#include namespace DB { @@ -94,7 +95,6 @@ namespace ErrorCodes * TODO: Interpreter resources * TODO: Support max streams * TODO: Support GROUPINS SETS, const aggregation keys, overflow row - * TODO: Simplify buildings SETS for IN function * TODO: Support interpolate, LIMIT BY. * TODO: Support ORDER BY read in order optimization * TODO: Support GROUP BY read in order optimization @@ -1064,6 +1064,8 @@ void Planner::buildQueryPlanIfNeeded() CollectSourceColumnsVisitor collect_source_columns_visitor(data); collect_source_columns_visitor.visit(query_tree); + collectSets(query_tree, *planner_context); + query_plan = buildQueryPlanForJoinTreeNode(query_node.getJoinTree(), select_query_info, select_query_options, planner_context); ActionsChain actions_chain; diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 0641a182ca8..fabea27370d 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -327,49 +327,10 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma const auto & function_node = node->as(); auto in_first_argument = function_node.getArguments().getNodes().at(0); auto in_second_argument = function_node.getArguments().getNodes().at(1); - auto in_second_argument_node_type = in_second_argument->getNodeType(); - auto set_source_hash = in_second_argument->getTreeHash(); - String set_key = "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); - auto prepared_set = planner_context->getGlobalPlannerContext()->getSet(set_key); - - if (!prepared_set) - { - if (in_second_argument_node_type == QueryTreeNodeType::QUERY || - in_second_argument_node_type == QueryTreeNodeType::UNION) - { - const auto & settings = planner_context->getQueryContext()->getSettingsRef(); - SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - bool tranform_null_in = settings.transform_null_in; - - auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); - - planner_context->getGlobalPlannerContext()->registerSet(set_key, set); - planner_context->getGlobalPlannerContext()->registerSubqueryNodeForSet(set_key, SubqueryNodeForSet{in_second_argument, set}); - - prepared_set = std::move(set); - } - else if (in_second_argument_node_type == QueryTreeNodeType::CONSTANT) - { - auto & in_second_argument_constant_node = in_second_argument->as(); - - const auto & settings = planner_context->getQueryContext()->getSettingsRef(); - - auto set = makeSetForConstantValue( - in_first_argument->getResultType(), - in_second_argument_constant_node.getResultType(), - in_second_argument_constant_node.getConstantValue(), - settings); - - planner_context->getGlobalPlannerContext()->registerSet(set_key, set); - prepared_set = std::move(set); - } - else - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Function IN is supported only if second argument is constant or table expression"); - } - } + const auto & global_planner_context = planner_context->getGlobalPlannerContext(); + auto set_key = global_planner_context->getSetKey(in_second_argument.get()); + auto prepared_set = global_planner_context->getSetOrThrow(set_key); auto column_set = ColumnSet::create(1, std::move(prepared_set)); auto column_set_const = ColumnConst::create(std::move(column_set), 1); diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 94cd98fa87a..a03a632e2ef 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -21,13 +21,14 @@ using PlannerContextPtr = std::shared_ptr; /** Planner actions visitor is responsible for adding necessary actions to calculate query tree expression node * into actions dag. * - * Column name to identifier map in planner context must be already initialized. + * Preconditions: + * 1. Column name to identifier map in planner context must be already initialized. * Identifiers in this map are used as action dag node names for column query tree nodes. + * 2. Sets for IN functions are already collected in global context. * * During actions build, there is special handling for following functions: * 1. Aggregate functions are added in actions dag as INPUT nodes. Aggregate functions arguments are not added. - * 2. For function `in` and its variants, planner context is populated with necessary table expressions to compute for sets, - * and prepared sets. + * 2. For function `in` and its variants, already collected sets from global context are used. */ class PlannerActionsVisitor { diff --git a/src/Planner/PlannerCollectSets.cpp b/src/Planner/PlannerCollectSets.cpp new file mode 100644 index 00000000000..4669ea7db59 --- /dev/null +++ b/src/Planner/PlannerCollectSets.cpp @@ -0,0 +1,94 @@ +#include + +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +class CollectSetsMatcher +{ +public: + using Visitor = ConstInDepthQueryTreeVisitor; + + struct Data + { + const PlannerContext & planner_context; + }; + + static void visit(const QueryTreeNodePtr & node, Data & data) + { + auto * function_node = node->as(); + if (!function_node || !isNameOfInFunction(function_node->getFunctionName())) + return; + + auto in_first_argument = function_node->getArguments().getNodes().at(0); + auto in_second_argument = function_node->getArguments().getNodes().at(1); + auto in_second_argument_node_type = in_second_argument->getNodeType(); + + const auto & planner_context = data.planner_context; + const auto & global_planner_context = planner_context.getGlobalPlannerContext(); + const auto & settings = planner_context.getQueryContext()->getSettingsRef(); + + String set_key = global_planner_context->getSetKey(in_second_argument.get()); + auto prepared_set = global_planner_context->getSetOrNull(set_key); + + if (prepared_set) + return; + + if (in_second_argument_node_type == QueryTreeNodeType::QUERY || + in_second_argument_node_type == QueryTreeNodeType::UNION) + { + SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; + bool tranform_null_in = settings.transform_null_in; + + auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); + + global_planner_context->registerSet(set_key, set); + global_planner_context->registerSubqueryNodeForSet(set_key, SubqueryNodeForSet{in_second_argument, set}); + } + else if (in_second_argument_node_type == QueryTreeNodeType::CONSTANT) + { + auto & in_second_argument_constant_node = in_second_argument->as(); + + auto set = makeSetForConstantValue( + in_first_argument->getResultType(), + in_second_argument_constant_node.getResultType(), + in_second_argument_constant_node.getConstantValue(), + settings); + + global_planner_context->registerSet(set_key, std::move(set)); + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Function IN is supported only if second argument is constant or table expression"); + } + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) + { + return child_node->getNodeType() != QueryTreeNodeType::QUERY; + } +}; + +using CollectSetsVisitor = CollectSetsMatcher::Visitor; + +} + +void collectSets(const QueryTreeNodePtr & node, const PlannerContext & planner_context) +{ + CollectSetsVisitor::Data data {planner_context}; + CollectSetsVisitor visitor(data); + visitor.visit(node); +} + +} diff --git a/src/Planner/PlannerCollectSets.h b/src/Planner/PlannerCollectSets.h new file mode 100644 index 00000000000..a641d12f9d9 --- /dev/null +++ b/src/Planner/PlannerCollectSets.h @@ -0,0 +1,15 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +/** Collect prepared sets and sets for subqueries that are necessary to execute IN function and its variations. + * Collected sets are registered in global planner context. + */ +void collectSets(const QueryTreeNodePtr & node, const PlannerContext & planner_context); + +} diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 37e208ce8f8..a227af83b9c 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -10,6 +10,52 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +GlobalPlannerContext::SetKey GlobalPlannerContext::getSetKey(const IQueryTreeNode * set_source_node) const +{ + auto set_source_hash = set_source_node->getTreeHash(); + return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); +} + +void GlobalPlannerContext::registerSet(const SetKey & key, SetPtr set) +{ + set_key_to_set.emplace(key, std::move(set)); +} + +SetPtr GlobalPlannerContext::getSetOrNull(const SetKey & key) const +{ + auto it = set_key_to_set.find(key); + if (it == set_key_to_set.end()) + return nullptr; + + return it->second; +} + +SetPtr GlobalPlannerContext::getSetOrThrow(const SetKey & key) const +{ + auto it = set_key_to_set.find(key); + if (it == set_key_to_set.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "No set is registered for key {}", + key); + + return it->second; +} + +void GlobalPlannerContext::registerSubqueryNodeForSet(const SetKey & key, SubqueryNodeForSet subquery_node_for_set) +{ + auto node_type = subquery_node_for_set.subquery_node->getNodeType(); + if (node_type != QueryTreeNodeType::QUERY && + node_type != QueryTreeNodeType::UNION) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Invalid node for set table expression. Expected query or union. Actual {}", + subquery_node_for_set.subquery_node->formatASTForErrorMessage()); + if (!subquery_node_for_set.set) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Set must be initialized"); + + set_key_to_subquery_node.emplace(key, std::move(subquery_node_for_set)); +} + PlannerContext::PlannerContext(ContextPtr query_context_, GlobalPlannerContextPtr global_planner_context_) : query_context(std::move(query_context_)) , global_planner_context(std::move(global_planner_context_)) diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index c85a37a1570..c57027248cd 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -110,49 +110,45 @@ private: ColumnNameToColumnIdentifier column_name_to_column_identifier; }; +/// Subquery node for set struct SubqueryNodeForSet { QueryTreeNodePtr subquery_node; SetPtr set; }; +/** Global planner context contains common objects that are shared between each planner context. + * + * 1. Prepared sets. + * 2. Subqueries for sets. + */ class GlobalPlannerContext { public: GlobalPlannerContext() = default; + using SetKey = std::string; using SetKeyToSet = std::unordered_map; using SetKeyToSubqueryNode = std::unordered_map; - void registerSet(const String & key, const SetPtr & set) - { - set_key_to_set.emplace(key, set); - } + /// Get set key for query node + SetKey getSetKey(const IQueryTreeNode * set_source_node) const; - SetPtr getSet(const String & key) const - { - auto it = set_key_to_set.find(key); - if (it == set_key_to_set.end()) - return nullptr; + /// Register set for set key + void registerSet(const SetKey & key, SetPtr set); - return it->second; - } + /// Get set for key, if no set is registered null is returned + SetPtr getSetOrNull(const SetKey & key) const; - void registerSubqueryNodeForSet(const String & key, const SubqueryNodeForSet & subquery_node_for_set) - { - auto node_type = subquery_node_for_set.subquery_node->getNodeType(); - if (node_type != QueryTreeNodeType::QUERY && - node_type != QueryTreeNodeType::UNION) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Invalid node for set table expression. Expected query or union. Actual {}", - subquery_node_for_set.subquery_node->formatASTForErrorMessage()); - if (!subquery_node_for_set.set) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Set must be initialized"); + /// Get set for key, if no set is registered logical exception is throwed + SetPtr getSetOrThrow(const SetKey & key) const; - set_key_to_subquery_node.emplace(key, subquery_node_for_set); - } + /** Register subquery node for set + * Subquery node for set node must have QUERY or UNION type and set must be initialized. + */ + void registerSubqueryNodeForSet(const SetKey & key, SubqueryNodeForSet subquery_node_for_set); + /// Get subquery nodes for sets const SetKeyToSubqueryNode & getSubqueryNodesForSets() const { return set_key_to_subquery_node; From efc2433347b9cff53f48d2f0fd7745fa70c2a4d3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 27 Aug 2022 16:32:50 +0200 Subject: [PATCH 080/188] Added IN function Set table support --- src/Analyzer/QueryAnalysisPass.cpp | 8 +++++-- src/Analyzer/QueryAnalysisPass.h | 2 +- src/Planner/PlannerActionsVisitor.cpp | 4 ++-- src/Planner/PlannerCollectSets.cpp | 13 ++++++++++- .../02377_analyzer_in_function_set.reference | 2 ++ .../02377_analyzer_in_function_set.sql | 23 +++++++++++++++++++ 6 files changed, 46 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02377_analyzer_in_function_set.reference create mode 100644 tests/queries/0_stateless/02377_analyzer_in_function_set.sql diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 763760f7c52..5b9171af3aa 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -53,6 +53,7 @@ #include #include +#include #include #include @@ -2358,7 +2359,11 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc auto * query_node = in_second_argument->as(); auto * union_node = in_second_argument->as(); - if (table_node || table_function_node) + if (table_node && dynamic_cast(table_node->getStorage().get()) != nullptr) + { + /// If table is already prepared set, we do not replace it with subquery + } + else if (table_node || table_function_node) { const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); @@ -2412,7 +2417,6 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc function_lambda_arguments_indexes.push_back(function_argument_index); } else if (is_special_function_in && (function_argument->getNodeType() == QueryTreeNodeType::TABLE || - function_argument->getNodeType() == QueryTreeNodeType::TABLE_FUNCTION || function_argument->getNodeType() == QueryTreeNodeType::QUERY || function_argument->getNodeType() == QueryTreeNodeType::UNION)) { diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index 7da4d9f2971..4ab77f35c88 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -35,7 +35,7 @@ namespace DB * * For function `in` and its variations arguments are resolved, but sets are not build. * If left and right arguments are constants constant folding is performed. - * If right argument resolved as table function, or table, it is replaced with query that read only ordinary columns from underlying + * If right argument resolved as table function, or table, and table is not of type Set, it is replaced with query that read only ordinary columns from underlying * storage. * Example: SELECT id FROM test_table WHERE id IN test_table_other; * Result: SELECT id FROM test_table WHERE id IN (SELECT test_table_column FROM test_table_other); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index fabea27370d..ce29c2df38b 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -508,7 +508,7 @@ String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerCon for (size_t i = 0; i < function_parameters_nodes_size; ++i) { const auto & function_parameter_node = function_parameters_nodes[i]; - calculateActionsDAGNodeName(function_parameter_node.get(), planner_context); + calculateActionsDAGNodeName(function_parameter_node.get(), planner_context, node_to_name); if (i + 1 != function_parameters_nodes_size) buffer << ", "; @@ -525,7 +525,7 @@ String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerCon for (size_t i = 0; i < function_arguments_nodes_size; ++i) { const auto & function_argument_node = function_arguments_nodes[i]; - buffer << calculateActionsDAGNodeName(function_argument_node.get(), planner_context); + buffer << calculateActionsDAGNodeName(function_argument_node.get(), planner_context, node_to_name); if (i + 1 != function_arguments_nodes_size) buffer << ", "; diff --git a/src/Planner/PlannerCollectSets.cpp b/src/Planner/PlannerCollectSets.cpp index 4669ea7db59..a9f0313a0ed 100644 --- a/src/Planner/PlannerCollectSets.cpp +++ b/src/Planner/PlannerCollectSets.cpp @@ -2,11 +2,14 @@ #include +#include + #include #include #include #include #include +#include namespace DB { @@ -44,7 +47,15 @@ public: if (prepared_set) return; - if (in_second_argument_node_type == QueryTreeNodeType::QUERY || + /// Tables and table functions are replaced with subquery at Analysis stage, except special Set table. + auto * second_argument_table = in_second_argument->as(); + StorageSet * storage_set = second_argument_table != nullptr ? dynamic_cast(second_argument_table->getStorage().get()) : nullptr; + + if (storage_set) + { + global_planner_context->registerSet(set_key, storage_set->getSet()); + } + else if (in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION) { SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; diff --git a/tests/queries/0_stateless/02377_analyzer_in_function_set.reference b/tests/queries/0_stateless/02377_analyzer_in_function_set.reference new file mode 100644 index 00000000000..b32da0d591a --- /dev/null +++ b/tests/queries/0_stateless/02377_analyzer_in_function_set.reference @@ -0,0 +1,2 @@ +0 Value_0 +1 Value_1 diff --git a/tests/queries/0_stateless/02377_analyzer_in_function_set.sql b/tests/queries/0_stateless/02377_analyzer_in_function_set.sql new file mode 100644 index 00000000000..e5c27e72ea1 --- /dev/null +++ b/tests/queries/0_stateless/02377_analyzer_in_function_set.sql @@ -0,0 +1,23 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value_0'), (1, 'Value_1'), (2, 'Value_2'); + +DROP TABLE IF EXISTS special_set_table; +CREATE TABLE special_set_table +( + id UInt64 +) ENGINE=Set; + +INSERT INTO special_set_table VALUES (0), (1); + +SELECT id, value FROM test_table WHERE id IN special_set_table; + +DROP TABLE special_set_table; +DROP TABLE test_table; From 2b571004f3c5bdfb1dd6d1f7d3d11255f07528fa Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 31 Aug 2022 17:21:17 +0200 Subject: [PATCH 081/188] Added support for projection names --- src/Analyzer/ArrayJoinNode.cpp | 2 +- src/Analyzer/ColumnNode.cpp | 8 - src/Analyzer/ColumnNode.h | 18 - src/Analyzer/ConstantNode.cpp | 26 +- src/Analyzer/ConstantNode.h | 17 +- src/Analyzer/ConstantValue.h | 47 + src/Analyzer/FunctionNode.cpp | 35 +- src/Analyzer/FunctionNode.h | 13 + src/Analyzer/IQueryTreeNode.h | 32 +- src/Analyzer/Identifier.h | 37 +- src/Analyzer/JoinNode.cpp | 4 +- src/Analyzer/LambdaNode.cpp | 5 - src/Analyzer/LambdaNode.h | 5 + src/Analyzer/MatcherNode.h | 6 + src/Analyzer/QueryAnalysisPass.cpp | 1208 +++++++++++++---- src/Analyzer/QueryAnalysisPass.h | 5 +- src/Analyzer/QueryNode.cpp | 67 +- src/Analyzer/QueryNode.h | 35 +- src/Analyzer/SetUtils.cpp | 4 +- src/Analyzer/SetUtils.h | 4 +- src/Analyzer/UnionNode.cpp | 33 +- src/Analyzer/UnionNode.h | 25 +- src/Analyzer/Utils.cpp | 58 +- src/Analyzer/Utils.h | 5 +- src/Interpreters/InterpreterDescribeQuery.cpp | 18 +- .../InterpreterSelectQueryAnalyzer.cpp | 12 + .../InterpreterSelectQueryAnalyzer.h | 5 + src/Planner/Planner.cpp | 134 +- src/Planner/PlannerActionsVisitor.cpp | 77 +- src/Planner/PlannerActionsVisitor.h | 6 +- src/Planner/PlannerCollectSets.cpp | 27 +- src/Planner/PlannerContext.cpp | 34 +- src/Planner/PlannerContext.h | 31 +- src/Planner/PlannerSorting.cpp | 17 +- .../02338_analyzer_constants_basic.reference | 4 +- .../02339_analyzer_matcher_basic.reference | 8 +- .../02340_analyzer_functions.reference | 2 +- .../02342_analyzer_compound_types.reference | 80 +- ...lyzer_column_transformers_strict.reference | 1 + .../02378_analyzer_projection_names.reference | 258 ++++ .../02378_analyzer_projection_names.sql | 215 +++ 41 files changed, 1990 insertions(+), 638 deletions(-) create mode 100644 src/Analyzer/ConstantValue.h create mode 100644 tests/queries/0_stateless/02378_analyzer_projection_names.reference create mode 100644 tests/queries/0_stateless/02378_analyzer_projection_names.sql diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp index 074acc2e8af..73323c7d369 100644 --- a/src/Analyzer/ArrayJoinNode.cpp +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -57,7 +57,7 @@ ASTPtr ArrayJoinNode::toASTImpl() const array_join_ast->expression_list = array_join_ast->children.back(); ASTPtr tables_in_select_query_ast = std::make_shared(); - addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[table_expression_child_index]); + addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[table_expression_child_index]); auto array_join_query_element_ast = std::make_shared(); array_join_query_element_ast->children.push_back(std::move(array_join_ast)); diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index 07cb4e377d8..f62d94bda14 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -101,8 +101,6 @@ void ColumnNode::updateTreeHashImpl(HashState & hash_state) const QueryTreeNodePtr ColumnNode::cloneImpl() const { auto clone_result = std::make_shared(column, column_source); - clone_result->display_identifier = display_identifier; - return clone_result; } @@ -118,12 +116,6 @@ void ColumnNode::getPointersToUpdateAfterClone(QueryTreePointersToUpdate & point ASTPtr ColumnNode::toASTImpl() const { - if (!display_identifier.empty()) - { - std::vector parts = display_identifier.getParts(); - return std::make_shared(std::move(parts)); - } - return std::make_shared(column.name); } diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index 2d0894628d4..62d53733588 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -17,12 +17,6 @@ namespace DB * * During query analysis pass identifier node is resolved into column. See IdentifierNode.h. * - * It is also important for client provide display identifier for AST conversion in case of JOINS are used. - * Example: SELECT t1.id, t2.id FROM test_table_1 AS t1, test_table_2 AS t2. - * In this example t1.id will be column with name id, but during conversion to AST, to keep AST valid it is important - * to save column name qualification. Display identifier does not take part in implementation of tree hash, - * or tree equals operator. It is only used during AST conversion. - * * Examples: * SELECT id FROM test_table. id is identifier that must be resolved to column node during query analysis pass. * SELECT lambda(x -> x + 1, [1,2,3]). x is identifier inside lambda that must be resolved to column node during query analysis pass. @@ -56,17 +50,6 @@ public: return column.name; } - /// Set display identifier that will be used during convertion to AST - void setDisplayIdentifier(const Identifier & display_identifier_value) - { - display_identifier = display_identifier_value; - } - - const Identifier & getDisplayIdentifier() const - { - return display_identifier; - } - /// Get column type const DataTypePtr & getColumnType() const { @@ -149,7 +132,6 @@ protected: private: NameAndTypePair column; QueryTreeNodeWeakPtr column_source; - Identifier display_identifier; static constexpr size_t expression_child_index = 0; static constexpr size_t children_size = expression_child_index + 1; diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 92d9b56d5cf..c187bf71ae5 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -14,16 +14,18 @@ namespace DB { +ConstantNode::ConstantNode(ConstantValuePtr constant_value_) + : constant_value(std::move(constant_value_)) + , value_string_dump(applyVisitor(FieldVisitorToString(), constant_value->getValue())) +{ +} + ConstantNode::ConstantNode(Field value_, DataTypePtr value_data_type_) - : value(std::move(value_)) - , value_string_dump(applyVisitor(FieldVisitorToString(), value)) - , type(std::move(value_data_type_)) + : ConstantNode(std::make_shared(std::move(value_), std::move(value_data_type_))) {} ConstantNode::ConstantNode(Field value_) - : value(std::move(value_)) - , value_string_dump(applyVisitor(FieldVisitorToString(), value)) - , type(applyVisitor(FieldToDataType(), value)) + : ConstantNode(std::make_shared(value_, applyVisitor(FieldToDataType(), value_))) {} void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const @@ -33,19 +35,19 @@ void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state if (hasAlias()) buffer << ", alias: " << getAlias(); - buffer << ", value: " << value.dump(); - buffer << ", result_type: " << type->getName(); + buffer << ", constant_value: " << constant_value->getValue().dump(); + buffer << ", constant_value_type: " << constant_value->getType()->getName(); } bool ConstantNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); - return value == rhs_typed.value && value_string_dump == rhs_typed.value_string_dump && type->equals(*rhs_typed.type); + return *constant_value == *rhs_typed.constant_value && value_string_dump == rhs_typed.value_string_dump; } void ConstantNode::updateTreeHashImpl(HashState & hash_state) const { - auto type_name = type->getName(); + auto type_name = constant_value->getType()->getName(); hash_state.update(type_name.size()); hash_state.update(type_name); @@ -55,12 +57,12 @@ void ConstantNode::updateTreeHashImpl(HashState & hash_state) const ASTPtr ConstantNode::toASTImpl() const { - return std::make_shared(value); + return std::make_shared(constant_value->getValue()); } QueryTreeNodePtr ConstantNode::cloneImpl() const { - return std::make_shared(value, type); + return std::make_shared(constant_value); } } diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index 02fd6a2c65f..41a58a4caf9 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -17,6 +17,9 @@ using ConstantNodePtr = std::shared_ptr; class ConstantNode final : public IQueryTreeNode { public: + /// Construct constant query tree node from constant value + explicit ConstantNode(ConstantValuePtr constant_value_); + /// Construct constant query tree node from field and data type explicit ConstantNode(Field value_, DataTypePtr value_data_type_); @@ -24,9 +27,14 @@ public: explicit ConstantNode(Field value_); /// Get constant value - const Field & getConstantValue() const + const Field & getValue() const { - return value; + return constant_value->getValue(); + } + + ConstantValuePtr getConstantValueOrNull() const override + { + return constant_value; } QueryTreeNodeType getNodeType() const override @@ -41,7 +49,7 @@ public: DataTypePtr getResultType() const override { - return type; + return constant_value->getType(); } protected: @@ -56,9 +64,8 @@ protected: QueryTreeNodePtr cloneImpl() const override; private: - Field value; + ConstantValuePtr constant_value; String value_string_dump; - DataTypePtr type; }; } diff --git a/src/Analyzer/ConstantValue.h b/src/Analyzer/ConstantValue.h new file mode 100644 index 00000000000..a9e2ffd9e65 --- /dev/null +++ b/src/Analyzer/ConstantValue.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/** Immutable constant value representation during analysis stage. + * Some query nodes can be represented by constant (scalar subqueries, functions with constant arguments). + */ +class ConstantValue; +using ConstantValuePtr = std::shared_ptr; + +class ConstantValue +{ +public: + ConstantValue(Field value_, DataTypePtr data_type_) + : value(std::move(value_)) + , data_type(std::move(data_type_)) + {} + + const Field & getValue() const + { + return value; + } + + const DataTypePtr & getType() const + { + return data_type; + } +private: + Field value; + DataTypePtr data_type; +}; + +inline bool operator==(const ConstantValue & lhs, const ConstantValue & rhs) +{ + return lhs.getValue() == rhs.getValue() && lhs.getType()->equals(*rhs.getType()); +} + +inline bool operator!=(const ConstantValue & lhs, const ConstantValue & rhs) +{ + return !(lhs == rhs); +} + +} diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index 518bb9709d6..2e31d2da6c8 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -1,7 +1,7 @@ #include -#include #include +#include #include #include @@ -46,11 +46,17 @@ void FunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state buffer << ", alias: " << getAlias(); buffer << ", function_name: " << function_name; - buffer << ", function_type: " << (function ? "ordinary_function" : "aggregate_function"); + buffer << ", is_aggregate_function: " << isAggregateFunction(); if (result_type) buffer << ", result_type: " + result_type->getName(); + if (constant_value) + { + buffer << ", constant_value: " << constant_value->getValue().dump(); + buffer << ", constant_value_type: " << constant_value->getType()->getName(); + } + const auto & parameters = getParameters(); if (!parameters.getNodes().empty()) { @@ -95,14 +101,21 @@ bool FunctionNode::isEqualImpl(const IQueryTreeNode & rhs) const isNonAggregateFunction() != rhs_typed.isNonAggregateFunction()) return false; - if (!result_type && !rhs_typed.result_type) - return true; + if (result_type && rhs_typed.result_type && !result_type->equals(*rhs_typed.getResultType())) + return false; else if (result_type && !rhs_typed.result_type) return false; else if (!result_type && rhs_typed.result_type) return false; - return result_type->equals(*rhs_typed.result_type); + if (constant_value && rhs_typed.constant_value && *constant_value != *rhs_typed.constant_value) + return false; + else if (constant_value && !rhs_typed.constant_value) + return false; + else if (!constant_value && rhs_typed.constant_value) + return false; + + return true; } void FunctionNode::updateTreeHashImpl(HashState & hash_state) const @@ -117,6 +130,17 @@ void FunctionNode::updateTreeHashImpl(HashState & hash_state) const hash_state.update(result_type_name.size()); hash_state.update(result_type_name); } + + if (constant_value) + { + auto constant_dump = applyVisitor(FieldVisitorToString(), constant_value->getValue()); + hash_state.update(constant_dump.size()); + hash_state.update(constant_dump); + + auto constant_value_type_name = constant_value->getType()->getName(); + hash_state.update(constant_value_type_name.size()); + hash_state.update(constant_value_type_name); + } } ASTPtr FunctionNode::toASTImpl() const @@ -146,6 +170,7 @@ QueryTreeNodePtr FunctionNode::cloneImpl() const result_function->function = function; result_function->aggregate_function = aggregate_function; result_function->result_type = result_type; + result_function->constant_value = constant_value; return result_function; } diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index 7ffb2af5c5d..d80231b7ab7 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -141,6 +142,17 @@ public: */ void resolveAsAggregateFunction(AggregateFunctionPtr aggregate_function_value, DataTypePtr result_type_value); + /// Perform constant folding for function node + void performConstantFolding(ConstantValuePtr constant_folded_value) + { + constant_value = std::move(constant_folded_value); + } + + ConstantValuePtr getConstantValueOrNull() const override + { + return constant_value; + } + QueryTreeNodeType getNodeType() const override { return QueryTreeNodeType::FUNCTION; @@ -172,6 +184,7 @@ private: FunctionOverloadResolverPtr function; AggregateFunctionPtr aggregate_function; DataTypePtr result_type; + ConstantValuePtr constant_value; }; } diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 38be6af5300..3356fdb7b31 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -11,6 +11,7 @@ #include #include +#include class SipHash; @@ -85,6 +86,29 @@ public: throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method getResultType is not supported for {} query node", getNodeTypeName()); } + /// Returns true if node has constant value + bool hasConstantValue() const + { + return getConstantValueOrNull() != nullptr; + } + + /** Returns constant value with type if node has constant value, and can be replaced with it. + * Examples: scalar subquery, function with constant arguments. + */ + virtual const ConstantValue & getConstantValue() const + { + auto constant_value = getConstantValueOrNull(); + if (!constant_value) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Node does not have constant value"); + + return *constant_value; + } + + virtual ConstantValuePtr getConstantValueOrNull() const + { + return {}; + } + /// Dump query tree to string String dumpTree() const; @@ -114,25 +138,25 @@ public: /// Get a deep copy of the query tree QueryTreeNodePtr clone() const; - /// Check if node has alias + /// Returns true if node has alias, false otherwise bool hasAlias() const { return !alias.empty(); } - /// Get node alias value if specified + /// Get node alias const String & getAlias() const { return alias; } - /// Set node alias value + /// Set node alias void setAlias(String alias_value) { alias = std::move(alias_value); } - /// Remove node alias value + /// Remove node alias void removeAlias() { alias = {}; diff --git a/src/Analyzer/Identifier.h b/src/Analyzer/Identifier.h index 9d9404f6831..d799060f8cf 100644 --- a/src/Analyzer/Identifier.h +++ b/src/Analyzer/Identifier.h @@ -22,14 +22,14 @@ public: /// Create Identifier from parts explicit Identifier(const std::vector & parts_) : parts(parts_) - , full_name(boost::algorithm::join(parts_, ".")) + , full_name(boost::algorithm::join(parts, ".")) { } /// Create Identifier from parts explicit Identifier(std::vector && parts_) : parts(std::move(parts_)) - , full_name(boost::algorithm::join(parts_, ".")) + , full_name(boost::algorithm::join(parts, ".")) { } @@ -154,9 +154,12 @@ public: assert(parts_to_remove_size <= parts.size()); for (size_t i = 0; i < parts_to_remove_size; ++i) + { + size_t last_part_size = parts.back().size(); parts.pop_back(); - - full_name = boost::algorithm::join(parts, "."); + bool is_not_last = !parts.empty(); + full_name.resize(full_name.size() - (last_part_size + static_cast(is_not_last))); + } } void popLast() @@ -164,6 +167,32 @@ public: return popLast(1); } + void pop_back() /// NOLINT + { + popLast(); + } + + void push_back(std::string && part) /// NOLINT + { + parts.push_back(std::move(part)); + full_name += '.'; + full_name += parts.back(); + } + + void push_back(const std::string & part) /// NOLINT + { + parts.push_back(part); + full_name += '.'; + full_name += parts.back(); + } + + template + void emplace_back(Args&&... args) /// NOLINT + { + parts.emplace_back(std::forward(args)...); + full_name += '.'; + full_name += parts.back(); + } private: std::vector parts; std::string full_name; diff --git a/src/Analyzer/JoinNode.cpp b/src/Analyzer/JoinNode.cpp index 5227b82c119..e723bbc2d9a 100644 --- a/src/Analyzer/JoinNode.cpp +++ b/src/Analyzer/JoinNode.cpp @@ -122,7 +122,7 @@ ASTPtr JoinNode::toASTImpl() const { ASTPtr tables_in_select_query_ast = std::make_shared(); - addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[left_table_expression_child_index]); + addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[left_table_expression_child_index]); size_t join_table_index = tables_in_select_query_ast->children.size(); @@ -141,7 +141,7 @@ ASTPtr JoinNode::toASTImpl() const join_ast->on_expression = std::move(join_expression_ast); } - addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[right_table_expression_child_index]); + addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[right_table_expression_child_index]); auto & table_element = tables_in_select_query_ast->children.at(join_table_index)->as(); table_element.children.push_back(std::move(join_ast)); diff --git a/src/Analyzer/LambdaNode.cpp b/src/Analyzer/LambdaNode.cpp index 52778b30c5d..5e9f8a1861f 100644 --- a/src/Analyzer/LambdaNode.cpp +++ b/src/Analyzer/LambdaNode.cpp @@ -29,16 +29,11 @@ LambdaNode::LambdaNode(Names argument_names_, QueryTreeNodePtr expression_) void LambdaNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { - auto result_type = getExpression()->getResultType(); - buffer << std::string(indent, ' ') << "LAMBDA id: " << format_state.getNodeId(this); if (hasAlias()) buffer << ", alias: " << getAlias(); - if (result_type) - buffer << ", result_type: " << result_type->getName(); - const auto & arguments = getArguments(); if (!arguments.getNodes().empty()) { diff --git a/src/Analyzer/LambdaNode.h b/src/Analyzer/LambdaNode.h index dc005f2b120..bbd2035acc5 100644 --- a/src/Analyzer/LambdaNode.h +++ b/src/Analyzer/LambdaNode.h @@ -95,6 +95,11 @@ public: return getExpression()->getResultType(); } + ConstantValuePtr getConstantValueOrNull() const override + { + return getExpression()->getConstantValueOrNull(); + } + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; bool isEqualImpl(const IQueryTreeNode & rhs) const override; diff --git a/src/Analyzer/MatcherNode.h b/src/Analyzer/MatcherNode.h index f21c759c328..010d6e44e23 100644 --- a/src/Analyzer/MatcherNode.h +++ b/src/Analyzer/MatcherNode.h @@ -102,6 +102,12 @@ public: return !qualified_identifier.empty(); } + /// Returns true if matcher is not qualified with identifier, false otherwise + bool isUnqualified() const + { + return qualified_identifier.empty(); + } + /// Get qualified identifier const Identifier & getQualifiedIdentifier() const { diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 5b9171af3aa..79354bcb32f 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -92,6 +92,7 @@ namespace ErrorCodes extern const int AMBIGUOUS_IDENTIFIER; extern const int INVALID_WITH_FILL_EXPRESSION; extern const int INVALID_LIMIT_EXPRESSION; + extern const int EMPTY_LIST_OF_COLUMNS_QUERIED; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -182,6 +183,8 @@ namespace ErrorCodes * TODO: JOIN support SELF JOIN with MergeTree. JOIN support matchers. * TODO: WINDOW functions * TODO: Table expression modifiers final, sample_size, sample_offset + * TODO: Scalar subqueries subquery depth + * TODO: Support function identifier resolve from parent query scope, if lambda in parent scope does not capture any columns. */ namespace @@ -360,8 +363,12 @@ struct IdentifierResolveSettings bool allow_to_check_database_catalog = true; }; -struct TableExpressionColumns +struct TableExpressionData { + std::string table_expression_name; + std::string table_expression_description; + std::string table_name; + std::string database_name; std::unordered_map column_name_to_column_node; std::unordered_set column_identifier_first_parts; @@ -531,6 +538,29 @@ struct IdentifierResolveScope /// Use identifier lookup to result cache bool use_identifier_lookup_to_result_cache = true; + /// Table expression node to data cache + std::unordered_map table_expression_node_to_data; + + /// Stage when names for projection are calculated + bool projection_names_calculation_stage = false; + + /// Node to projection name + std::unordered_map node_to_projection_name; + + TableExpressionData & getTableExpressionDataOrThrow(QueryTreeNodePtr table_expression_node) + { + auto it = table_expression_node_to_data.find(table_expression_node); + if (it == table_expression_node_to_data.end()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Table expression {} data must be initialized. In scope {}", + table_expression_node->formatASTForErrorMessage(), + scope_node->formatASTForErrorMessage()); + } + + return it->second; + } + /// Dump identifier resolve scope [[maybe_unused]] void dump(WriteBuffer & buffer) const { @@ -637,6 +667,14 @@ public: updateAliasesIfNeeded(data, child, false); return false; } + else if (auto * union_node = child->as()) + { + if (union_node->isCTE()) + return false; + + updateAliasesIfNeeded(data, child, false); + return false; + } return true; } @@ -800,7 +838,9 @@ private: QueryTreeNodePtr tryResolveIdentifierFromTableColumns(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); - QueryTreeNodePtr tryResolveIdentifierFromTable(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + bool tryBindIdentifierToTableExpression(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + + QueryTreeNodePtr tryResolveIdentifierFromTableExpression(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); QueryTreeNodePtr tryResolveIdentifierFromJoin(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); @@ -816,6 +856,14 @@ private: /// Resolve query tree nodes functions + using QueryTreeNodesWithNames = std::vector>; + + void matcherQualifyColumnsForProjectionNamesIfNeeded(QueryTreeNodesWithNames & matched_nodes_with_column_names, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + + QueryTreeNodesWithNames resolveQualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); + + QueryTreeNodesWithNames resolveUnqualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); + QueryTreeNodePtr resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); void resolveLambda(const QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope); @@ -828,6 +876,10 @@ private: void resolveSortColumnsNodeList(QueryTreeNodePtr & sort_columns_node_list, IdentifierResolveScope & scope); + String calculateProjectionNodeDisplayName(QueryTreeNodePtr & node, IdentifierResolveScope & scope); + + NamesAndTypes resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope); + void initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope); void initializeTableExpressionColumns(QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); @@ -844,9 +896,6 @@ private: /// Lambdas that are currently in resolve process std::unordered_set lambdas_in_resolve_process; - /// Table expression node to columns cache - std::unordered_map table_expression_node_to_columns; - /// Array join expressions counter size_t array_join_expressions_counter = 0; }; @@ -892,11 +941,17 @@ QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunction(const std return result_node; } -/** Evaluate scalar subquery. - * In result of this function node will be replaced by constant node. - */ +/// Evaluate scalar subquery and perform constant folding. void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node) { + auto * query_node = node->as(); + auto * union_node = node->as(); + if (!query_node && !union_node) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Node must have query or union type. Actual {} {}", + node->getNodeTypeName(), + node->formatASTForErrorMessage()); + auto subquery_context = Context::createCopy(context); Settings subquery_settings = context->getSettings(); subquery_settings.max_result_rows = 1; @@ -911,9 +966,9 @@ void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node) } size_t subquery_depth = 0; - auto options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth + 1, true); + auto options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth + 1, true /*is_subqueyr*/); - auto interpreter = std::make_unique(node->toAST(), options, subquery_context); + auto interpreter = std::make_unique(node, options, subquery_context); auto io = interpreter->execute(); @@ -941,7 +996,13 @@ void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node) type = makeNullable(type); } - node = std::make_shared(Null()); + auto constant_value = std::make_shared(Null(), std::move(type)); + + if (query_node) + query_node->performConstantFolding(std::move(constant_value)); + else if (union_node) + query_node->performConstantFolding(std::move(constant_value)); + return; } @@ -985,9 +1046,11 @@ void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node) scalar_type = std::make_shared(block.getDataTypes()); } - auto original_ast = node->getOriginalAST(); - node = std::make_shared(std::move(scalar_value), std::move(scalar_type)); - node->setOriginalAST(std::move(original_ast)); + auto constant_value = std::make_shared(std::move(scalar_value), std::move(scalar_type)); + if (query_node) + query_node->performConstantFolding(std::move(constant_value)); + else if (union_node) + union_node->performConstantFolding(std::move(constant_value)); } /// Resolve identifier functions implementation @@ -1173,10 +1236,14 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier scope.expressions_in_resolve_process_stack.pushNode(it->second); + auto node_type = it->second->getNodeType(); + /// Resolve expression if necessary - if (auto * alias_identifier = it->second->as()) + if (node_type == QueryTreeNodeType::IDENTIFIER) { - auto lookup_result = tryResolveIdentifier(IdentifierLookup{alias_identifier->getIdentifier(), identifier_lookup.lookup_context}, scope, identifier_resolve_settings); + auto & alias_identifier_node = it->second->as(); + auto identifier = alias_identifier_node.getIdentifier(); + auto lookup_result = tryResolveIdentifier(IdentifierLookup{identifier, identifier_lookup.lookup_context}, scope, identifier_resolve_settings); it->second = lookup_result.resolved_identifier; /** During collection of aliases if node is identifier and has alias, we cannot say if it is @@ -1192,11 +1259,11 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier else if (identifier_lookup.isFunctionLookup() && it->second) scope.alias_name_to_expression_node.erase(identifier_bind_part); } - else if (auto * function = it->second->as()) + else if (node_type == QueryTreeNodeType::FUNCTION) { resolveFunction(it->second, scope); } - else if (auto * query = it->second->as()) + else if (node_type == QueryTreeNodeType::QUERY) { IdentifierResolveScope subquery_scope(it->second, &scope /*parent_scope*/); resolveQuery(it->second, subquery_scope); @@ -1204,6 +1271,14 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier if (identifier_lookup.isExpressionLookup()) evaluateScalarSubquery(it->second); } + else if (node_type == QueryTreeNodeType::UNION) + { + IdentifierResolveScope subquery_scope(it->second, &scope /*parent_scope*/); + resolveUnion(it->second, subquery_scope); + + if (identifier_lookup.isExpressionLookup()) + evaluateScalarSubquery(it->second); + } scope.expressions_in_resolve_process_stack.popNode(); @@ -1270,14 +1345,14 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableColumns(const Ident return result; } -QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) +bool QueryAnalyzer::tryBindIdentifierToTableExpression(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) { - auto * table_node = table_expression_node->as(); - auto * query_node = table_expression_node->as(); - auto * union_node = table_expression_node->as(); - auto * table_function_node = table_expression_node->as(); + auto table_expression_node_type = table_expression_node->getNodeType(); - if (!table_node && !table_function_node && !query_node && !union_node) + if (table_expression_node_type != QueryTreeNodeType::TABLE && + table_expression_node_type != QueryTreeNodeType::TABLE_FUNCTION && + table_expression_node_type != QueryTreeNodeType::QUERY && + table_expression_node_type != QueryTreeNodeType::UNION) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Unexpected table expression. Expected table, table function, query or union node. Actual {}. In scope {}", table_expression_node->formatASTForErrorMessage(), @@ -1286,30 +1361,10 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo const auto & identifier = identifier_lookup.identifier; const auto & path_start = identifier.getParts().front(); - std::string table_expression_name; + auto & table_expression_data = scope.getTableExpressionDataOrThrow(table_expression_node); - std::string table_name; - std::string database_name; - - if (table_node) - { - const auto & table_storage_id = table_node->getStorageID(); - table_name = table_storage_id.table_name; - database_name = table_storage_id.database_name; - table_expression_name = table_storage_id.getFullNameNotQuoted(); - } - else if (query_node || union_node) - { - table_name = query_node ? query_node->getCTEName() : union_node->getCTEName(); - - if (table_expression_node->hasAlias()) - table_expression_name = table_expression_node->getAlias(); - } - else if (table_function_node) - { - if (table_function_node->hasAlias()) - table_expression_name = table_function_node->getAlias(); - } + const auto & table_name = table_expression_data.table_name; + const auto & database_name = table_expression_data.database_name; if (identifier_lookup.isTableLookup()) { @@ -1318,7 +1373,63 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected identifier {} to contain 1 or 2 parts size to be resolved as table. In scope {}", identifier_lookup.identifier.getFullName(), - table_node->formatASTForErrorMessage()); + table_expression_node->formatASTForErrorMessage()); + + if (parts_size == 1 && path_start == table_name) + return true; + else if (parts_size == 2 && path_start == database_name && identifier[1] == table_name) + return true; + else + return false; + } + + if (table_expression_data.hasFullIdentifierName(IdentifierView(identifier)) || table_expression_data.canBindIdentifier(IdentifierView(identifier))) + return true; + + if (identifier.getPartsSize() == 1) + return false; + + if ((!table_name.empty() && path_start == table_name) || (table_expression_node->hasAlias() && path_start == table_expression_node->getAlias())) + return true; + + if (identifier.getPartsSize() == 2) + return false; + + if (!database_name.empty() && path_start == database_name && identifier[1] == table_name) + return true; + + return false; +} + +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) +{ + auto table_expression_node_type = table_expression_node->getNodeType(); + + if (table_expression_node_type != QueryTreeNodeType::TABLE && + table_expression_node_type != QueryTreeNodeType::TABLE_FUNCTION && + table_expression_node_type != QueryTreeNodeType::QUERY && + table_expression_node_type != QueryTreeNodeType::UNION) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Unexpected table expression. Expected table, table function, query or union node. Actual {}. In scope {}", + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + const auto & identifier = identifier_lookup.identifier; + const auto & path_start = identifier.getParts().front(); + + auto & table_expression_data = scope.getTableExpressionDataOrThrow(table_expression_node); + + if (identifier_lookup.isTableLookup()) + { + size_t parts_size = identifier_lookup.identifier.getPartsSize(); + if (parts_size != 1 && parts_size != 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Expected identifier {} to contain 1 or 2 parts size to be resolved as table. In scope {}", + identifier_lookup.identifier.getFullName(), + table_expression_node->formatASTForErrorMessage()); + + const auto & table_name = table_expression_data.table_name; + const auto & database_name = table_expression_data.database_name; if (parts_size == 1 && path_start == table_name) return table_expression_node; @@ -1328,14 +1439,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo return {}; } - auto storage_columns_it = table_expression_node_to_columns.find(table_expression_node.get()); - if (storage_columns_it == table_expression_node_to_columns.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Table expression {} columns must be initialized. In scope {}", - table_expression_node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - - auto & storage_columns = storage_columns_it->second; auto resolve_identifier_from_storage_or_throw = [&](size_t identifier_column_qualifier_parts) -> QueryTreeNodePtr { auto identifier_view = IdentifierView(identifier); @@ -1354,17 +1457,17 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo bool compound_identifier = identifier_view.getPartsSize() > 1; bool match_full_identifier = false; - auto it = storage_columns.column_name_to_column_node.find(std::string(identifier_view.getFullName())); - if (it != storage_columns.column_name_to_column_node.end()) + auto it = table_expression_data.column_name_to_column_node.find(std::string(identifier_view.getFullName())); + if (it != table_expression_data.column_name_to_column_node.end()) { match_full_identifier = true; result_column = it->second; } else { - it = storage_columns.column_name_to_column_node.find(std::string(identifier_view.at(0))); + it = table_expression_data.column_name_to_column_node.find(std::string(identifier_view.at(0))); - if (it != storage_columns.column_name_to_column_node.end()) + if (it != table_expression_data.column_name_to_column_node.end()) result_column = it->second; } @@ -1372,14 +1475,12 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo throw Exception(ErrorCodes::BAD_ARGUMENTS, "Identifier {} cannot be resolved from {}{}. In scope {}", identifier.getFullName(), - query_node ? "subquery" : "table", - table_expression_name.empty() ? "" : " with name " + table_expression_name, + table_expression_data.table_expression_description, + table_expression_data.table_expression_name.empty() ? "" : " with name " + table_expression_data.table_expression_name, scope.scope_node->formatASTForErrorMessage()); - if (identifier_column_qualifier_parts > 0) - result_column->setDisplayIdentifier(identifier); - QueryTreeNodePtr result_expression = result_column; + bool projection_name_clone_is_needed = true; if (!match_full_identifier && compound_identifier) { @@ -1387,6 +1488,53 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo nested_path.popFirst(); auto tuple_element_result = wrapExpressionNodeInTupleElement(result_expression, identifier_view); resolveFunction(tuple_element_result, scope); + result_expression = std::move(tuple_element_result); + projection_name_clone_is_needed = false; + } + + if (scope.projection_names_calculation_stage && scope.table_expression_node_to_data.size() > 1) + { + if (projection_name_clone_is_needed) + result_expression = result_expression->clone(); + + auto qualified_identifier = identifier; + for (size_t i = 0; i < identifier_column_qualifier_parts; ++i) + { + auto qualified_identifier_with_removed_part = qualified_identifier; + qualified_identifier_with_removed_part.popFirst(); + + if (qualified_identifier_with_removed_part.empty()) + break; + + if (context->getSettingsRef().prefer_column_name_to_alias + && scope.alias_name_to_expression_node.contains(qualified_identifier_with_removed_part[0])) + break; + + bool can_remove_qualificator = true; + + for (auto & table_expression_to_check_data : scope.table_expression_node_to_data) + { + const auto & table_expression_to_check = table_expression_to_check_data.first; + if (table_expression_to_check.get() == table_expression_node.get()) + continue; + + IdentifierLookup column_identifier_lookup {qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION}; + bool can_bind_identifier_to_table_expression = tryBindIdentifierToTableExpression(column_identifier_lookup, table_expression_to_check, scope); + if (can_bind_identifier_to_table_expression) + { + can_remove_qualificator = false; + break; + } + } + + if (!can_remove_qualificator) + break; + + qualified_identifier = std::move(qualified_identifier_with_removed_part); + } + + auto qualified_identifier_full_name = qualified_identifier.getFullName(); + scope.node_to_projection_name.emplace(result_expression, std::move(qualified_identifier_full_name)); } return result_expression; @@ -1398,19 +1546,23 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLo * Storage alias works for subquery, table function as well. * 3. Try to bind identifier first parts to database name and table name, if true remove first two parts and try to get full identifier from table or throw exception. */ - if (storage_columns.canBindIdentifier(IdentifierView(identifier)) || - storage_columns.hasFullIdentifierName(IdentifierView(identifier))) + if (table_expression_data.hasFullIdentifierName(IdentifierView(identifier))) + return resolve_identifier_from_storage_or_throw(0 /*identifier_column_qualifier_parts*/); + + if (table_expression_data.canBindIdentifier(IdentifierView(identifier))) return resolve_identifier_from_storage_or_throw(0 /*identifier_column_qualifier_parts*/); if (identifier.getPartsSize() == 1) return {}; + const auto & table_name = table_expression_data.table_name; if ((!table_name.empty() && path_start == table_name) || (table_expression_node->hasAlias() && path_start == table_expression_node->getAlias())) return resolve_identifier_from_storage_or_throw(1 /*identifier_column_qualifier_parts*/); if (identifier.getPartsSize() == 2) return {}; + const auto & database_name = table_expression_data.database_name; if (!database_name.empty() && path_start == database_name && identifier[1] == table_name) return resolve_identifier_from_storage_or_throw(2 /*identifier_column_qualifier_parts*/); @@ -1550,8 +1702,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromArrayJoin(const Identifi */ if (!scope.table_expressions_in_resolve_process.contains(table_expression_node.get()) && resolved_identifier) { - auto & resolved_identifier_column = resolved_identifier->as(); - for (const auto & array_join_expression : from_array_join_node.getJoinExpressions().getNodes()) { auto & array_join_column_expression = array_join_expression->as(); @@ -1564,7 +1714,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromArrayJoin(const Identifi { auto array_join_column = array_join_column_expression.getColumn(); auto result = std::make_shared(array_join_column, table_expression_node); - result->setDisplayIdentifier(resolved_identifier_column.getDisplayIdentifier()); return result; } @@ -1599,7 +1748,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTreeNode(const Ident if (scope.table_expressions_in_resolve_process.contains(join_tree_node.get())) return {}; - return tryResolveIdentifierFromTable(identifier_lookup, join_tree_node, scope); + return tryResolveIdentifierFromTableExpression(identifier_lookup, join_tree_node, scope); } default: { @@ -1697,10 +1846,15 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const if (identifier_lookup.isTableLookup() && !is_cte) continue; - if (resolved_identifier->as()) + if (is_cte) + { return lookup_result; - else if (is_cte) + } + else if (const auto constant_value = resolved_identifier->getConstantValueOrNull()) + { + lookup_result.resolved_identifier = std::make_shared(constant_value); return lookup_result; + } throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Resolve identifier {} from parent scope only supported for constants and CTE. Actual {} node type {}. In scope {}", @@ -1855,145 +2009,232 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook /// Resolve query tree nodes functions implementation -/** Resolve query tree matcher. Check MatcherNode.h for detailed matcher description. Check ColumnTransformers.h for detailed transformers description. - * - * 1. Populate matcher_expression_nodes. - * - * If we resolve qualified matcher, first try to match qualified identifier to expression. If qualified identifier matched expression node then - * if expression is compound match it column names using matcher `isMatchingColumn` method, if expression is not compound, throw exception. - * If qualified identifier did not match expression in query tree, try to lookup qualified identifier in table context. - * - * If we resolve non qualified matcher, use current scope join tree node. - * - * 2. Apply column transformers to matched expression nodes. For strict column transformers save used column names. - * 3. Validate strict column transformers. - */ -QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope) + +void QueryAnalyzer::matcherQualifyColumnsForProjectionNamesIfNeeded(QueryTreeNodesWithNames & matched_nodes_with_column_names, + const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) { - auto & matcher_node_typed = matcher_node->as(); + if (!scope.projection_names_calculation_stage || scope.table_expression_node_to_data.size() < 2) + return; - std::vector matcher_expression_nodes; + std::vector qualify_identifier_parts; - if (matcher_node_typed.isQualified()) + if (table_expression_node->hasAlias()) + qualify_identifier_parts = {table_expression_node->getAlias()}; + else if (auto * table_node = table_expression_node->as()) + qualify_identifier_parts = {table_node->getStorageID().getDatabaseName(), table_node->getStorageID().getTableName()}; + + size_t qualify_identifier_parts_size = qualify_identifier_parts.size(); + + for (auto & [column_node, column_name] : matched_nodes_with_column_names) { - auto expression_identifier_lookup = IdentifierLookup{matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::EXPRESSION}; - auto expression_identifier_resolve_result = tryResolveIdentifier(expression_identifier_lookup, scope); - auto expression_query_tree_node = expression_identifier_resolve_result.resolved_identifier; + std::vector column_qualified_identifier_parts = Identifier(column_name).getParts(); - if (expression_query_tree_node) + for (size_t i = 0; i < qualify_identifier_parts_size; ++i) { - auto result_type = expression_query_tree_node->getResultType(); + bool need_to_qualify = false; + auto identifier_to_check = Identifier(column_qualified_identifier_parts); - while (const auto * array_type = typeid_cast(result_type.get())) - result_type = array_type->getNestedType(); - - const auto * tuple_data_type = typeid_cast(result_type.get()); - if (!tuple_data_type) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Qualified matcher {} find non compound expression {} with type {}. Expected tuple or array of tuples. In scope {}", - matcher_node->formatASTForErrorMessage(), - expression_query_tree_node->formatASTForErrorMessage(), - expression_query_tree_node->getResultType()->getName(), - scope.scope_node->formatASTForErrorMessage()); - - const auto & element_names = tuple_data_type->getElementNames(); - - for (const auto & element_name : element_names) + for (auto & table_expression_data : scope.table_expression_node_to_data) { - if (!matcher_node_typed.isMatchingColumn(element_name)) + if (table_expression_data.first.get() == table_expression_node.get()) continue; - auto tuple_element_function = std::make_shared("tupleElement"); - tuple_element_function->getArguments().getNodes().push_back(expression_query_tree_node); - tuple_element_function->getArguments().getNodes().push_back(std::make_shared(element_name)); - - QueryTreeNodePtr function_query_node = tuple_element_function; - resolveFunction(function_query_node, scope); - matcher_expression_nodes.push_back(std::move(function_query_node)); - } - } - else - { - IdentifierResolveSettings identifier_resolve_settings; - identifier_resolve_settings.allow_to_check_parent_scopes = false; - identifier_resolve_settings.allow_to_check_cte = false; - identifier_resolve_settings.allow_to_check_database_catalog = false; - - auto table_identifier_lookup = IdentifierLookup{matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::TABLE}; - auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope, identifier_resolve_settings); - auto table_expression_node = table_identifier_resolve_result.resolved_identifier; - - if (!table_expression_node || table_identifier_resolve_result.resolve_place == IdentifierResolvePlace::DATABASE_CATALOG) - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Qualified matcher {} does not find table. In scope {}", - matcher_node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); + IdentifierLookup lookup{identifier_to_check, IdentifierLookupContext::EXPRESSION}; + if (tryBindIdentifierToTableExpression(lookup, table_expression_data.first, scope)) + { + need_to_qualify = true; + break; + } } - NamesAndTypesList initial_matcher_columns; - - auto * table_expression_query_node = table_expression_node->as(); - auto * table_expression_table_node = table_expression_node->as(); - auto * table_expression_table_function_node = table_expression_node->as(); - - if (table_expression_query_node) + if (need_to_qualify) { - initial_matcher_columns = table_expression_query_node->computeProjectionColumns(); + size_t part_index_to_use_for_qualification = qualify_identifier_parts_size - i - 1; + const auto & part_to_use = qualify_identifier_parts[part_index_to_use_for_qualification]; + column_qualified_identifier_parts.insert(column_qualified_identifier_parts.begin(), part_to_use); } else { - const auto & storage_snapshot = table_expression_table_node ? table_expression_table_node->getStorageSnapshot() : table_expression_table_function_node->getStorageSnapshot(); - initial_matcher_columns = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All)); - } - - for (auto & column : initial_matcher_columns) - { - const auto & column_name = column.name; - if (matcher_node_typed.isMatchingColumn(column_name)) - matcher_expression_nodes.push_back(std::make_shared(column, table_expression_node)); + break; } } + + scope.node_to_projection_name.emplace(column_node, Identifier(column_qualified_identifier_parts).getFullName()); + } +} + +/** Resolve qualified tree matcher. + * + * First try to match qualified identifier to expression. If qualified identifier matched expression node then + * if expression is compound match it column names using matcher `isMatchingColumn` method, if expression is not compound, throw exception. + * If qualified identifier did not match expression in query tree, try to lookup qualified identifier in table context. + */ +QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope) +{ + auto & matcher_node_typed = matcher_node->as(); + assert(matcher_node_typed.isQualified()); + + QueryTreeNodesWithNames matched_expression_nodes_with_column_names; + + auto expression_identifier_lookup = IdentifierLookup{matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::EXPRESSION}; + auto expression_identifier_resolve_result = tryResolveIdentifier(expression_identifier_lookup, scope); + auto expression_query_tree_node = expression_identifier_resolve_result.resolved_identifier; + + /// Try to resolve unqualified matcher for query expression + + if (expression_query_tree_node) + { + auto result_type = expression_query_tree_node->getResultType(); + + while (const auto * array_type = typeid_cast(result_type.get())) + result_type = array_type->getNestedType(); + + const auto * tuple_data_type = typeid_cast(result_type.get()); + if (!tuple_data_type) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Qualified matcher {} find non compound expression {} with type {}. Expected tuple or array of tuples. In scope {}", + matcher_node->formatASTForErrorMessage(), + expression_query_tree_node->formatASTForErrorMessage(), + expression_query_tree_node->getResultType()->getName(), + scope.scope_node->formatASTForErrorMessage()); + + const auto & element_names = tuple_data_type->getElementNames(); + + auto matcher_qualified_identifier_copy = matcher_node_typed.getQualifiedIdentifier(); + for (const auto & element_name : element_names) + { + if (!matcher_node_typed.isMatchingColumn(element_name)) + continue; + + auto tuple_element_function = std::make_shared("tupleElement"); + tuple_element_function->getArguments().getNodes().push_back(expression_query_tree_node); + tuple_element_function->getArguments().getNodes().push_back(std::make_shared(element_name)); + + QueryTreeNodePtr function_query_node = tuple_element_function; + resolveFunction(function_query_node, scope); + + if (scope.projection_names_calculation_stage) + { + matcher_qualified_identifier_copy.push_back(element_name); + scope.node_to_projection_name.emplace(function_query_node, matcher_qualified_identifier_copy.getFullName()); + matcher_qualified_identifier_copy.pop_back(); + } + + matched_expression_nodes_with_column_names.emplace_back(std::move(function_query_node), element_name); + } + + return matched_expression_nodes_with_column_names; + } + + /// Try to resolve unqualified matcher for table expression + + IdentifierResolveSettings identifier_resolve_settings; + identifier_resolve_settings.allow_to_check_cte = false; + identifier_resolve_settings.allow_to_check_database_catalog = false; + + auto table_identifier_lookup = IdentifierLookup{matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::TABLE}; + auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope, identifier_resolve_settings); + auto table_expression_node = table_identifier_resolve_result.resolved_identifier; + + if (!table_expression_node) + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Qualified matcher {} does not find table. In scope {}", + matcher_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + + NamesAndTypes initial_matcher_columns; + + auto * table_expression_query_node = table_expression_node->as(); + auto * table_expression_union_node = table_expression_node->as(); + auto * table_expression_table_node = table_expression_node->as(); + auto * table_expression_table_function_node = table_expression_node->as(); + + if (table_expression_query_node || table_expression_union_node) + { + initial_matcher_columns = table_expression_query_node ? table_expression_query_node->getProjectionColumns() + : table_expression_union_node->computeProjectionColumns(); + } + else if (table_expression_table_node || table_expression_table_function_node) + { + const auto & storage_snapshot = table_expression_table_node ? table_expression_table_node->getStorageSnapshot() + : table_expression_table_function_node->getStorageSnapshot(); + auto storage_columns_list = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All)); + initial_matcher_columns = NamesAndTypes(storage_columns_list.begin(), storage_columns_list.end()); } else { - /** There can be edge case if matcher is inside lambda expression. - * Try to find parent query expression using parent scopes. - */ - IQueryTreeNode * scope_node = scope.scope_node.get(); - auto * scope_query_node = scope_node->as(); + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Invalid table expression node {}. In scope {}", + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } - while (!scope_query_node) + for (auto & column : initial_matcher_columns) + { + const auto & column_name = column.name; + if (matcher_node_typed.isMatchingColumn(column_name)) + matched_expression_nodes_with_column_names.emplace_back(std::make_shared(column, table_expression_node), column_name); + } + + matcherQualifyColumnsForProjectionNamesIfNeeded(matched_expression_nodes_with_column_names, table_expression_node, scope); + + return matched_expression_nodes_with_column_names; +} + + +/// Resolve non qualified matcher, using scope join tree node. +QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope) +{ + auto & matcher_node_typed = matcher_node->as(); + assert(matcher_node_typed.isUnqualified()); + + QueryTreeNodesWithNames matched_expression_nodes_with_column_names; + + /** There can be edge case if matcher is inside lambda expression. + * Try to find parent query expression using parent scopes. + */ + IQueryTreeNode * scope_node = scope.scope_node.get(); + auto * scope_query_node = scope_node->as(); + + while (!scope_query_node) + { + if (!scope.parent_scope) + break; + + scope_node = scope.parent_scope->scope_node.get(); + scope_query_node = scope_node->as(); + } + + /// If there are no parent query scope or query scope does not have join tree + if (!scope_query_node || !scope_query_node->getJoinTree()) + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Unqualified matcher {} cannot be resolved. There are no table sources. In scope {}", + matcher_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + + auto table_expressions = extractTableExpressions(scope_query_node->getJoinTree()); + + for (auto & table_expression : table_expressions) + { + auto * table_node = table_expression->as(); + auto * table_function_node = table_expression->as(); + auto * query_node = table_expression->as(); + auto * union_node = table_expression->as(); + + NamesAndTypes table_expression_columns; + + if (query_node || union_node) { - if (!scope.parent_scope) - break; - - scope_node = scope.parent_scope->scope_node.get(); - scope_query_node = scope_node->as(); + table_expression_columns = query_node ? query_node->getProjectionColumns() : union_node->computeProjectionColumns(); } - - /// If there are no parent scope that has tables or query scope does not have FROM section - if (!scope_query_node || !scope_query_node->getJoinTree()) + else if (table_node || table_function_node) { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Unqualified matcher {} cannot be resolved. There are no table sources. In scope {}", - matcher_node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - } - - NamesAndTypesList initial_matcher_columns; - - auto * from_query_node = scope_query_node->getJoinTree()->as(); - auto * from_table_node = scope_query_node->getJoinTree()->as(); - auto * from_table_function_node = scope_query_node->getJoinTree()->as(); - - if (from_query_node) - { - initial_matcher_columns = from_query_node->computeProjectionColumns(); - } - else if (from_table_node || from_table_function_node) - { - const auto & storage_snapshot = from_table_node ? from_table_node->getStorageSnapshot() : from_table_function_node->getStorageSnapshot(); + const auto & storage_snapshot + = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); UInt8 get_column_options_kind = 0; @@ -2014,24 +2255,50 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, } auto get_columns_options = GetColumnsOptions(static_cast(get_column_options_kind)); - initial_matcher_columns = storage_snapshot->getColumns(get_columns_options); + auto storage_columns_list = storage_snapshot->getColumns(get_columns_options); + table_expression_columns = NamesAndTypes(storage_columns_list.begin(), storage_columns_list.end()); } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unqualified matcher {} resolve unexpected FROM section {}", + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unqualified matcher {} resolve unexpected table expression. In scope {}", matcher_node_typed.formatASTForErrorMessage(), scope_query_node->getJoinTree()->formatASTForErrorMessage()); } - for (auto & column : initial_matcher_columns) + for (auto & table_expression_column : table_expression_columns) { - const auto & column_name = column.name; - if (matcher_node_typed.isMatchingColumn(column_name)) - matcher_expression_nodes.push_back(std::make_shared(column, scope_query_node->getJoinTree())); + if (!matcher_node_typed.isMatchingColumn(table_expression_column.name)) + continue; + + auto matched_column_node = std::make_shared(table_expression_column, table_expression); + matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), table_expression_column.name); } + + matcherQualifyColumnsForProjectionNamesIfNeeded(matched_expression_nodes_with_column_names, table_expression, scope); } + return matched_expression_nodes_with_column_names; +} + + +/** Resolve query tree matcher. Check MatcherNode.h for detailed matcher description. Check ColumnTransformers.h for detailed transformers description. + * + * 1. Populate matched expression nodes resolving qualified or unqualified matcher. + * 2. Apply column transformers to matched expression nodes. For strict column transformers save used column names. + * 3. Validate strict column transformers. + */ +QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope) +{ + auto & matcher_node_typed = matcher_node->as(); + + QueryTreeNodesWithNames matched_expression_nodes_with_names; + + if (matcher_node_typed.isQualified()) + matched_expression_nodes_with_names = resolveQualifiedMatcher(matcher_node, scope); + else + matched_expression_nodes_with_names = resolveUnqualifiedMatcher(matcher_node, scope); + std::unordered_map> strict_transformer_to_used_column_names; auto add_strict_transformer_column_name = [&](const IColumnTransformerNode * transformer, const std::string & column_name) { @@ -2047,13 +2314,17 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, ListNodePtr list = std::make_shared(); - for (auto & node : matcher_expression_nodes) + for (auto & [node, column_name] : matched_expression_nodes_with_names) { + bool apply_transformer_was_used = false; + bool replace_transformer_was_used = false; + for (const auto & transformer : matcher_node_typed.getColumnTransformers().getNodes()) { if (auto * apply_transformer = transformer->as()) { const auto & expression_node = apply_transformer->getExpressionNode(); + apply_transformer_was_used = true; if (apply_transformer->getApplyTransformerType() == ApplyColumnTransformerType::LAMBDA) { @@ -2097,13 +2368,13 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, } else if (auto * except_transformer = transformer->as()) { + if (apply_transformer_was_used || replace_transformer_was_used) + break; - auto node_name = node->getName(); - - if (except_transformer->isColumnMatching(node_name)) + if (except_transformer->isColumnMatching(column_name)) { if (except_transformer->isStrict()) - add_strict_transformer_column_name(except_transformer, node_name); + add_strict_transformer_column_name(except_transformer, column_name); node = {}; break; @@ -2111,16 +2382,19 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, } else if (auto * replace_transformer = transformer->as()) { + if (apply_transformer_was_used || replace_transformer_was_used) + break; - auto node_name = node->getName(); - auto replace_expression = replace_transformer->findReplacementExpression(node_name); + replace_transformer_was_used = true; + + auto replace_expression = replace_transformer->findReplacementExpression(column_name); if (!replace_expression) continue; if (replace_transformer->isStrict()) - add_strict_transformer_column_name(replace_transformer, node_name); + add_strict_transformer_column_name(replace_transformer, column_name); - node = replace_expression; + node = replace_expression->clone(); resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } } @@ -2278,9 +2552,7 @@ void QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const Qu lambda.getArgumentsNode() = std::move(lambda_new_arguments); - /** Lambda body expression is resolved as standard query expression node. - * After that lambda is resolved, because its expression node is resolved. - */ + /// Lambda body expression is resolved as standard query expression node. resolveExpressionNode(lambda.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); lambdas_in_resolve_process.erase(lambda_node.get()); @@ -2331,17 +2603,18 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc auto & parameters_nodes = function_node.getParameters().getNodes(); parameters.reserve(parameters_nodes.size()); - for (auto & parameter : parameters_nodes) + for (auto & parameter_node : parameters_nodes) { - auto * constant_parameter = parameter->as(); - if (constant_parameter) + auto constant_value = parameter_node->getConstantValueOrNull(); + + if (!constant_value) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Parameter for function {} expected to be constant expression. Actual {}. In scope {}", + "Parameter for function {} expected to have constant value. Actual {}. In scope {}", function_node.getFunctionName(), - parameter->formatASTForErrorMessage(), + parameter_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); - parameters.push_back(constant_parameter); + parameters.push_back(constant_value->getValue()); } bool is_special_function_in = isNameOfInFunction(function_name); @@ -2367,15 +2640,26 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc { const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); + + size_t columns_to_select_size = columns_to_select.size(); + auto column_nodes_to_select = std::make_shared(); + column_nodes_to_select->getNodes().reserve(columns_to_select_size); + + NamesAndTypes projection_columns; + projection_columns.reserve(columns_to_select_size); for (auto & column : columns_to_select) - column_nodes_to_select->getNodes().push_back(std::make_shared(column, in_second_argument)); + { + column_nodes_to_select->getNodes().emplace_back(std::make_shared(column, in_second_argument)); + projection_columns.emplace_back(column.name, column.type); + } auto in_second_argument_query_node = std::make_shared(); in_second_argument_query_node->setIsSubquery(true); in_second_argument_query_node->getProjectionNode() = std::move(column_nodes_to_select); in_second_argument_query_node->getJoinTree() = std::move(in_second_argument); + in_second_argument_query_node->resolveProjectionColumns(std::move(projection_columns)); in_second_argument = std::move(in_second_argument_query_node); } @@ -2433,10 +2717,15 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc function_node.getFunctionName(), scope.scope_node->formatASTForErrorMessage()); - if (auto * constant_query_tree_node = function_argument->as()) - argument_column.column = argument_column.type->createColumnConst(1, constant_query_tree_node->getConstantValue()); + if (const auto constant_value = function_argument->getConstantValueOrNull()) + { + argument_column.column = constant_value->getType()->createColumnConst(1, constant_value->getValue()); + argument_column.type = constant_value->getType(); + } else + { all_arguments_constants = false; + } argument_types.push_back(argument_column.type); argument_columns.emplace_back(std::move(argument_column)); @@ -2508,6 +2797,10 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc QueryTreeNodePtr function_query_node = tuple_element_function; resolveFunction(function_query_node, scope); + + if (scope.projection_names_calculation_stage && node->hasAlias()) + scope.node_to_projection_name.emplace(function_query_node, node->getAlias() + '.' + element_name); + result_list->getNodes().push_back(std::move(function_query_node)); } @@ -2623,18 +2916,17 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc * Example: SELECT * FROM test_table LIMIT 1 IN 1; */ if (is_special_function_in && - function_arguments.at(0)->getNodeType() == QueryTreeNodeType::CONSTANT && - function_arguments.at(1)->getNodeType() == QueryTreeNodeType::CONSTANT) + function_arguments.at(0)->hasConstantValue() && + function_arguments.at(1)->hasConstantValue()) { - const auto & first_argument = assert_cast(*function_arguments[0]); - const auto & second_argument = assert_cast(*function_arguments[1]); + const auto & first_argument_constant_value = function_arguments[0]->getConstantValue(); + const auto & second_argument_constant_value = function_arguments[1]->getConstantValue(); - const auto & first_argument_type = first_argument.getResultType(); - const auto & second_argument_type = second_argument.getResultType(); + const auto & first_argument_constant_type = first_argument_constant_value.getType(); + const auto & second_argument_constant_literal = second_argument_constant_value.getValue(); + const auto & second_argument_constant_type = second_argument_constant_value.getType(); - const auto & second_argument_value = second_argument.getConstantValue(); - - auto set = makeSetForConstantValue(first_argument_type, second_argument_type, second_argument_value, context->getSettingsRef()); + auto set = makeSetForConstantValue(first_argument_constant_type, second_argument_constant_literal, second_argument_constant_type, context->getSettingsRef()); /// Create constant set column for constant folding @@ -2675,10 +2967,7 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc Field constant_value; column->get(0, constant_value); - auto original_ast = function_node.getOriginalAST(); - node = std::make_shared(std::move(constant_value), result_type); - node->setOriginalAST(std::move(original_ast)); - return; + function_node.performConstantFolding(std::make_shared(std::move(constant_value), result_type)); } } } @@ -2763,7 +3052,8 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes { auto & identifier_node = node->as(); auto unresolved_identifier = identifier_node.getIdentifier(); - node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::EXPRESSION}, scope).resolved_identifier; + auto resolve_identifier_expression_result = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::EXPRESSION}, scope); + node = resolve_identifier_expression_result.resolved_identifier; if (node && !node_alias.empty()) scope.alias_name_to_lambda_node.erase(node_alias); @@ -2776,8 +3066,6 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes scope.alias_name_to_expression_node.erase(node_alias); } - bool node_already_cloned = false; - if (!node && allow_table_expression) { node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::TABLE}, scope).resolved_identifier; @@ -2787,7 +3075,6 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes if (resolved_as_cte) { - node_already_cloned = true; node = node->clone(); node->as().setIsCTE(false); } @@ -2810,10 +3097,6 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes scope.scope_node->formatASTForErrorMessage()); } - if (!node_already_cloned) - node = node->clone(); - - node->setAlias(node_alias); break; } case QueryTreeNodeType::MATCHER: @@ -2961,29 +3244,28 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes */ void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) { - auto & initial_node_list = node_list->as(); - size_t initial_node_list_size = initial_node_list.getNodes().size(); + auto & node_list_typed = node_list->as(); + size_t node_list_size = node_list_typed.getNodes().size(); - auto result_node_list = std::make_shared(); - result_node_list->getNodes().reserve(initial_node_list_size); + QueryTreeNodes result_nodes; + result_nodes.reserve(node_list_size); - auto initial_node_list_nodes_copy = initial_node_list.getNodes(); - for (auto & node : initial_node_list_nodes_copy) + for (auto & node : node_list_typed.getNodes()) { resolveExpressionNode(node, scope, allow_lambda_expression, allow_table_expression); if (auto * expression_list = node->as()) { for (auto & expression_list_node : expression_list->getNodes()) - result_node_list->getNodes().push_back(std::move(expression_list_node)); + result_nodes.push_back(std::move(expression_list_node)); } else { - result_node_list->getNodes().push_back(std::move(node)); + result_nodes.push_back(std::move(node)); } } - node_list = std::move(result_node_list); + node_list_typed.getNodes() = std::move(result_nodes); } /** Resolve sort columns nodes list. @@ -3038,6 +3320,323 @@ void QueryAnalyzer::resolveSortColumnsNodeList(QueryTreeNodePtr & sort_columns_n } } +class SubqueryToProjectionNameMatcher +{ +public: + using Visitor = ConstInDepthQueryTreeVisitor; + + struct Data + { + std::unordered_map & node_to_display_name; + size_t subquery_index = 1; + }; + + static void visit(const QueryTreeNodePtr &, Data &) + { + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node, Data & data) + { + auto child_node_type = child_node->getNodeType(); + if (child_node_type != QueryTreeNodeType::QUERY && child_node_type != QueryTreeNodeType::UNION) + return true; + + data.node_to_display_name.emplace(child_node, "_subquery_" + std::to_string(data.subquery_index)); + ++data.subquery_index; + return false; + } +}; + +using SubqueryToProjectionNameVisitor = SubqueryToProjectionNameMatcher::Visitor; + +String QueryAnalyzer::calculateProjectionNodeDisplayName(QueryTreeNodePtr & node, IdentifierResolveScope & scope) +{ + if (node->hasAlias()) + return node->getAlias(); + + auto calculate_inner_expression_display_name = [&](QueryTreeNodePtr & expression_node, IdentifierResolveScope & node_scope) + { + auto * expression_function_node = expression_node->as(); + if (expression_node->getNodeType() == QueryTreeNodeType::MATCHER) + expression_node = resolveMatcher(expression_node, node_scope); + else if (expression_function_node && expression_function_node->getFunctionName() == "untuple") + resolveFunction(expression_node, node_scope); + + if (expression_node->getNodeType() == QueryTreeNodeType::LIST) + { + auto & list_nodes = expression_node->as().getNodes(); + size_t list_nodes_size = list_nodes.size(); + + WriteBufferFromOwnString result_buffer; + for (size_t i = 0; i < list_nodes_size; ++i) + { + auto & list_expression_node = list_nodes[i]; + String list_node_display_name = calculateProjectionNodeDisplayName(list_expression_node, node_scope); + result_buffer << list_node_display_name; + + if (i + 1 != list_nodes_size) + result_buffer << ", "; + } + + return result_buffer.str(); + } + + return calculateProjectionNodeDisplayName(expression_node, node_scope); + }; + + auto try_to_get_projection_name_from_scope = [&](const QueryTreeNodePtr & expression_node) + { + bool check_in_parent_query = scope.scope_node->getNodeType() != QueryTreeNodeType::QUERY; + IdentifierResolveScope * scope_to_check = &scope; + + while (scope_to_check != nullptr) + { + auto projection_name_it = scope_to_check->node_to_projection_name.find(expression_node); + if (projection_name_it != scope_to_check->node_to_projection_name.end()) + return projection_name_it->second; + + scope_to_check = scope_to_check->parent_scope; + if (scope_to_check && scope_to_check->scope_node->getNodeType() == QueryTreeNodeType::QUERY) + { + if (check_in_parent_query) + check_in_parent_query = false; + else + break; + } + } + + return String(); + }; + + auto projection_name_from_scope = try_to_get_projection_name_from_scope(node); + if (!projection_name_from_scope.empty()) + return projection_name_from_scope; + + auto node_type = node->getNodeType(); + if (node_type == QueryTreeNodeType::CONSTANT) + { + const auto & constant_node = node->as(); + return constant_node.getName(); + } + else if (node_type == QueryTreeNodeType::COLUMN) + { + const auto & column_node = node->as(); + const auto & column_name = column_node.getColumnName(); + return column_name; + } + else if (node_type == QueryTreeNodeType::IDENTIFIER) + { + const auto & identifier_node = node->as(); + auto identifier = identifier_node.getIdentifier(); + IdentifierLookup lookup {identifier, IdentifierLookupContext::EXPRESSION}; + auto resolved_identifier_result = tryResolveIdentifier(lookup, scope); + + if (resolved_identifier_result.resolved_identifier && resolved_identifier_result.isResolvedFromJoinTree()) + { + projection_name_from_scope = try_to_get_projection_name_from_scope(node); + std::cout << "Projection name from scope " << projection_name_from_scope << std::endl; + + if (!projection_name_from_scope.empty()) + return projection_name_from_scope; + if (auto * column_node = resolved_identifier_result.resolved_identifier->as()) + return column_node->getColumnName(); + } + + return identifier_node.getIdentifier().getFullName(); + } + else if (node_type == QueryTreeNodeType::MATCHER) + { + /// Top level matcher + return {}; + } + else if (node_type == QueryTreeNodeType::FUNCTION) + { + auto & function_node = node->as(); + + WriteBufferFromOwnString buffer; + buffer << function_node.getFunctionName(); + + auto & function_parameters_nodes = function_node.getParameters().getNodes(); + + if (!function_parameters_nodes.empty()) + { + buffer << '('; + + size_t function_parameters_nodes_size = function_parameters_nodes.size(); + for (size_t i = 0; i < function_parameters_nodes_size; ++i) + { + auto & function_parameter_node = function_parameters_nodes[i]; + String function_parameter_node_display_name = calculate_inner_expression_display_name(function_parameter_node, scope); + buffer << function_parameter_node_display_name; + + if (i + 1 != function_parameters_nodes_size) + buffer << ", "; + } + + buffer << ')'; + } + + buffer << '('; + + auto & function_arguments_nodes = function_node.getArguments().getNodes(); + size_t function_arguments_nodes_size = function_arguments_nodes.size(); + for (size_t i = 0; i < function_arguments_nodes_size; ++i) + { + auto & function_argument_node = function_arguments_nodes[i]; + String function_argument_node_display_name = calculate_inner_expression_display_name(function_argument_node, scope); + buffer << function_argument_node_display_name; + + if (i + 1 != function_arguments_nodes_size) + buffer << ", "; + } + + buffer << ')'; + + return buffer.str(); + } + else if (node_type == QueryTreeNodeType::LAMBDA) + { + auto & lambda_node = node->as(); + IdentifierResolveScope lambda_scope(node, &scope /*parent_scope*/); + lambda_scope.projection_names_calculation_stage = true; + String lambda_expression_display_name = calculate_inner_expression_display_name(lambda_node.getExpression(), lambda_scope); + + WriteBufferFromOwnString buffer; + buffer << "lambda(tuple("; + + const auto & lambda_argument_names = lambda_node.getArgumentNames(); + size_t lambda_argument_names_size = lambda_argument_names.size(); + + for (size_t i = 0; i < lambda_argument_names_size; ++i) + { + const auto & argument_name = lambda_argument_names[i]; + buffer << argument_name; + + if (i + 1 != lambda_argument_names_size) + buffer << ", "; + } + + buffer << "), "; + buffer << lambda_expression_display_name; + buffer << ')'; + + return buffer.str(); + } + else if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Subquery name must be already collected"); + } + else if (node_type == QueryTreeNodeType::TABLE) + { + /// Table node can be second argument of IN function + const auto & table_node = node->as(); + return table_node.getStorageID().getFullNameNotQuoted(); + } + else if (node_type == QueryTreeNodeType::TABLE_FUNCTION) + { + /// Table function node can be second argument of IN function + auto & table_function_node = node->as(); + + WriteBufferFromOwnString buffer; + buffer << table_function_node.getTableFunctionName(); + + buffer << '('; + + auto & table_function_arguments_nodes = table_function_node.getArguments().getNodes(); + size_t table_function_arguments_nodes_size = table_function_arguments_nodes.size(); + for (size_t i = 0; i < table_function_arguments_nodes_size; ++i) + { + auto & function_argument_node = table_function_arguments_nodes[i]; + if (auto * identifier_node = function_argument_node->as()) + { + if (identifier_node->hasAlias()) + buffer << identifier_node->getAlias(); + else + buffer << identifier_node->getIdentifier().getFullName(); + } + else + { + buffer << calculateProjectionNodeDisplayName(function_argument_node, scope); + } + + if (i + 1 != table_function_arguments_nodes_size) + buffer << ", "; + } + + buffer << ')'; + + return buffer.str(); + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Invalid projection node {} {}", node->getNodeType(), node->dumpTree()); + } +} + +NamesAndTypes QueryAnalyzer::resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope) +{ + auto & initial_node_list = projection_node_list->as(); + + scope.projection_names_calculation_stage = true; + + SubqueryToProjectionNameVisitor::Data subquery_to_projection_name_visitor_data {scope.node_to_projection_name}; + SubqueryToProjectionNameVisitor subquery_to_projection_name_visitor(subquery_to_projection_name_visitor_data); + subquery_to_projection_name_visitor.visit(projection_node_list); + + auto initial_node_list_nodes_copy = initial_node_list.getNodes(); + size_t list_nodes_copy_size = initial_node_list_nodes_copy.size(); + + std::vector> projection_nodes_with_display_name; + projection_nodes_with_display_name.reserve(list_nodes_copy_size); + + for (size_t i = 0; i < list_nodes_copy_size; ++i) + { + auto & node = initial_node_list_nodes_copy[i]; + + String display_name = calculateProjectionNodeDisplayName(node, scope); + resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + if (auto * expression_list = node->as()) + { + for (auto & expression_list_node : expression_list->getNodes()) + projection_nodes_with_display_name.emplace_back(expression_list_node, calculateProjectionNodeDisplayName(expression_list_node, scope)); + } + else + { + projection_nodes_with_display_name.emplace_back(node, display_name); + } + } + + size_t projection_nodes_size = projection_nodes_with_display_name.size(); + + auto projection_result_node_list = std::make_shared(); + NamesAndTypes projection_columns; + projection_columns.reserve(projection_nodes_size); + projection_result_node_list->getNodes().reserve(projection_nodes_size); + + for (size_t i = 0; i < projection_nodes_size; ++i) + { + auto && projection_node = projection_nodes_with_display_name[i].first; + + if (projection_node->getNodeType() != QueryTreeNodeType::CONSTANT && + projection_node->getNodeType() != QueryTreeNodeType::FUNCTION && + projection_node->getNodeType() != QueryTreeNodeType::COLUMN && + projection_node->getNodeType() != QueryTreeNodeType::QUERY && + projection_node->getNodeType() != QueryTreeNodeType::UNION) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Projection node must be constant, function, column, query or union"); + + projection_columns.emplace_back(projection_nodes_with_display_name[i].second, projection_node->getResultType()); + projection_result_node_list->getNodes().emplace_back(std::move(projection_node)); + } + + scope.projection_names_calculation_stage = false; + scope.node_to_projection_name.clear(); + + projection_node_list = std::move(projection_result_node_list); + return projection_columns; +} + /** Initialize query join tree node. * * 1. Resolve identifiers. @@ -3077,23 +3676,11 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod from_table_identifier.getIdentifier().getFullName(), scope.scope_node->formatASTForErrorMessage()); - /** SELECT a.id, b.id FROM test_table_join_1 AS a INNER JOIN test_table_join_1 AS b. - * Because of identifier cache we receive same result for same identifier. - * Use table_expressions_in_resolve_process to detect if resolved identifier must be cloned. - */ - if (scope.table_expressions_in_resolve_process.contains(resolved_identifier.get())) - { - resolved_identifier = resolved_identifier->clone(); - resolved_identifier->removeAlias(); - } - + resolved_identifier = resolved_identifier->clone(); bool resolved_as_cte = resolved_identifier->as() && resolved_identifier->as()->isCTE(); if (resolved_as_cte) - { - resolved_identifier = resolved_identifier->clone(); resolved_identifier->as().setIsCTE(false); - } auto current_join_tree_node_alias = current_join_tree_node->getAlias(); resolved_identifier->setAlias(current_join_tree_node_alias); @@ -3163,11 +3750,40 @@ void QueryAnalyzer::initializeTableExpressionColumns(QueryTreeNodePtr & table_ex table_expression_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); - auto storage_columns_it = table_expression_node_to_columns.find(table_expression_node.get()); - if (storage_columns_it != table_expression_node_to_columns.end()) + auto table_expression_data_it = scope.table_expression_node_to_data.find(table_expression_node); + if (table_expression_data_it != scope.table_expression_node_to_data.end()) return; - TableExpressionColumns storage_columns; + TableExpressionData table_expression_data; + + std::string table_expression_name; + std::string table_expression_description; + + std::string table_name; + std::string database_name; + + if (table_node) + { + const auto & table_storage_id = table_node->getStorageID(); + table_expression_data.table_name = table_storage_id.table_name; + table_expression_data.database_name = table_storage_id.database_name; + table_expression_data.table_expression_name = table_storage_id.getFullNameNotQuoted(); + table_expression_data.table_expression_description = "table"; + } + else if (query_node || union_node) + { + table_expression_data.table_name = query_node ? query_node->getCTEName() : union_node->getCTEName(); + table_expression_data.table_expression_description = "subquery"; + + if (table_expression_node->hasAlias()) + table_expression_data.table_expression_name = table_expression_node->getAlias(); + } + else if (table_function_node) + { + table_expression_data.table_expression_description = "table_function"; + if (table_function_node->hasAlias()) + table_expression_data.table_expression_name = table_function_node->getAlias(); + } if (table_node || table_function_node) { @@ -3224,29 +3840,29 @@ void QueryAnalyzer::initializeTableExpressionColumns(QueryTreeNodePtr & table_ex column_name_to_column_node[alias_column_to_resolve_name] = alias_column_to_resolve; } - storage_columns.column_name_to_column_node = std::move(column_name_to_column_node); + table_expression_data.column_name_to_column_node = std::move(column_name_to_column_node); } else if (query_node || union_node) { - auto column_names_and_types = query_node ? query_node->computeProjectionColumns() : union_node->computeProjectionColumns(); - storage_columns.column_name_to_column_node.reserve(column_names_and_types.size()); + auto column_names_and_types = query_node ? query_node->getProjectionColumns() : union_node->computeProjectionColumns(); + table_expression_data.column_name_to_column_node.reserve(column_names_and_types.size()); for (const auto & column_name_and_type : column_names_and_types) { auto column_node = std::make_shared(column_name_and_type, table_expression_node); - storage_columns.column_name_to_column_node.emplace(column_name_and_type.name, column_node); + table_expression_data.column_name_to_column_node.emplace(column_name_and_type.name, column_node); } } - storage_columns.column_identifier_first_parts.reserve(storage_columns.column_name_to_column_node.size()); + table_expression_data.column_identifier_first_parts.reserve(table_expression_data.column_name_to_column_node.size()); - for (auto & [column_name, _] : storage_columns.column_name_to_column_node) + for (auto & [column_name, _] : table_expression_data.column_name_to_column_node) { Identifier column_name_identifier(column_name); - storage_columns.column_identifier_first_parts.insert(column_name_identifier.at(0)); + table_expression_data.column_identifier_first_parts.insert(column_name_identifier.at(0)); } - table_expression_node_to_columns.emplace(table_expression_node.get(), std::move(storage_columns)); + scope.table_expression_node_to_data.emplace(table_expression_node, std::move(table_expression_data)); } /** Resolve query join tree. @@ -3299,8 +3915,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, { auto hints = TableFunctionFactory::instance().getHints(table_function_name); if (!hints.empty()) - throw Exception( - ErrorCodes::UNKNOWN_FUNCTION, + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}. Maybe you meant: {}", table_function_name, DB::toString(hints)); @@ -3355,28 +3970,27 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, auto & array_join_node = join_tree_node->as(); resolveQueryJoinTreeNode(array_join_node.getTableExpression(), scope, expressions_visitor); + /// Wrap array join expressions into column nodes, where array join expression is inner expression. + for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) { - auto array_join_expression_alias = array_join_expression->getAlias(); + if (array_join_expression->getNodeType() == QueryTreeNodeType::MATCHER) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "ARRAY JOIN matcher is not supported"); + auto array_join_expression_alias = array_join_expression->getAlias(); if (!array_join_expression_alias.empty() && scope.alias_name_to_expression_node.contains(array_join_expression_alias)) throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, "ARRAY JOIN expression {} with duplicate alias {}. In scope {}", array_join_expression->formatASTForErrorMessage(), array_join_expression_alias, scope.scope_node->formatASTForErrorMessage()); - } - /// Add array join expressions into scope + /// Add array join expression into scope + expressions_visitor.visit(array_join_expression); - expressions_visitor.visit(array_join_node.getJoinExpressionsNode()); - resolveExpressionNode(array_join_node.getJoinExpressionsNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + resolveExpressionNode(array_join_expression, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - /// Wrap array join expressions into column nodes, where array join expression is inner expression. - - for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) - { - auto array_join_expression_alias = array_join_expression->getAlias(); auto result_type = array_join_expression->getResultType(); if (!isArray(result_type)) @@ -3386,8 +4000,6 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, result_type->getName(), scope.scope_node->formatASTForErrorMessage()); - array_join_expression->removeAlias(); - result_type = assert_cast(*result_type).getNestedType(); auto array_join_expression_name = "__array_join_expression_" + std::to_string(array_join_expressions_counter); @@ -3597,6 +4209,12 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /// Resolve query node sections. + auto projection_columns = resolveProjectionExpressionNodeList(query_node_typed.getProjectionNode(), scope); + if (query_node_typed.getProjection().getNodes().empty()) + throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED, + "Empty list of columns in projection. In scope {}", + scope.scope_node->formatASTForErrorMessage()); + if (query_node_typed.hasWith()) resolveExpressionNodeList(query_node_typed.getWithNode(), scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); @@ -3616,14 +4234,14 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier { resolveExpressionNode(query_node_typed.getLimit(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - const auto * limit_constant_node = query_node_typed.getLimit()->as(); - if (!limit_constant_node || !isNativeNumber(limit_constant_node->getResultType())) + const auto limit_constant_value = query_node_typed.getLimit()->getConstantValueOrNull(); + if (!limit_constant_value || !isNativeNumber(removeNullable(limit_constant_value->getType()))) throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, "Limit expression must be constant with numeric type. Actual {}. In scope {}", query_node_typed.getLimit()->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); - Field converted = convertFieldToType(limit_constant_node->getConstantValue(), DataTypeUInt64()); + Field converted = convertFieldToType(limit_constant_value->getValue(), DataTypeUInt64()); if (converted.isNull()) throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, "Limit numeric constant expression is not representable as UInt64"); @@ -3633,21 +4251,19 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier { resolveExpressionNode(query_node_typed.getOffset(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - const auto * offset_constant_node = query_node_typed.getOffset()->as(); - if (!offset_constant_node || !isNativeNumber(offset_constant_node->getResultType())) + const auto offset_constant_value = query_node_typed.getOffset()->getConstantValueOrNull(); + if (!offset_constant_value || !isNativeNumber(removeNullable(offset_constant_value->getType()))) throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, "Offset expression must be constant with numeric type. Actual {}. In scope {}", query_node_typed.getLimit()->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); - Field converted = convertFieldToType(offset_constant_node->getConstantValue(), DataTypeUInt64()); + Field converted = convertFieldToType(offset_constant_value->getValue(), DataTypeUInt64()); if (converted.isNull()) throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, "Offset numeric constant expression is not representable as UInt64"); } - resolveExpressionNodeList(query_node_typed.getProjectionNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - /** WITH section can be safely removed, because WITH section only can provide aliases to query expressions * and CTE for other sections to use. * @@ -3704,7 +4320,17 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier node->formatASTForErrorMessage(), node_alias, scope.scope_node->formatASTForErrorMessage()); + + node->removeAlias(); } + + for (auto & [_, node] : scope.alias_name_to_expression_node) + node->removeAlias(); + + for (auto & [_, node] : scope.alias_name_to_lambda_node) + node->removeAlias(); + + query_node_typed.resolveProjectionColumns(std::move(projection_columns)); } void QueryAnalyzer::resolveUnion(const QueryTreeNodePtr & union_node, IdentifierResolveScope & scope) diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index 4ab77f35c88..9fc17e237a6 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -26,7 +26,10 @@ namespace DB * TODO: Scalar subqueries must be evaluated only if they are part of query tree where we must have constant. This is currently not done * because execution layer does not support scalar subqueries execution. * - * 8. Special functions handling: + * 8. For query node projection columns are calculated. Later passes cannot change type, display name of projection column, and cannot add or remove + * columns in projection section. + * + * 9. Special functions handling: * Function `untuple` is handled properly. * Function `arrayJoin` is handled properly. * diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 5253bb15333..1785f846fd3 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -1,6 +1,7 @@ #include #include +#include #include @@ -30,29 +31,6 @@ QueryNode::QueryNode() children[order_by_child_index] = std::make_shared(); } -NamesAndTypesList QueryNode::computeProjectionColumns() const -{ - NamesAndTypes query_columns; - - const auto & projection_nodes = getProjection(); - query_columns.reserve(projection_nodes.getNodes().size()); - - for (const auto & projection_node : projection_nodes.getNodes()) - { - auto column_type = projection_node->getResultType(); - std::string column_name; - - if (projection_node->hasAlias()) - column_name = projection_node->getAlias(); - else - column_name = projection_node->getName(); - - query_columns.emplace_back(column_name, column_type); - } - - return {query_columns.begin(), query_columns.end()}; -} - String QueryNode::getName() const { WriteBufferFromOwnString buffer; @@ -105,12 +83,33 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s if (!cte_name.empty()) buffer << ", cte_name: " << cte_name; + if (constant_value) + { + buffer << ", constant_value: " << constant_value->getValue().dump(); + buffer << ", constant_value_type: " << constant_value->getType()->getName(); + } + if (hasWith()) { buffer << '\n' << std::string(indent + 2, ' ') << "WITH\n"; getWith().dumpTreeImpl(buffer, format_state, indent + 4); } + if (!projection_columns.empty()) + { + buffer << '\n'; + buffer << std::string(indent + 2, ' ') << "PROJECTION COLUMNS\n"; + + size_t projection_columns_size = projection_columns.size(); + for (size_t i = 0; i < projection_columns_size; ++i) + { + const auto & projection_column = projection_columns[i]; + buffer << std::string(indent + 4, ' ') << projection_column.name << " " << projection_column.type->getName(); + if (i + 1 != projection_columns_size) + buffer << '\n'; + } + } + buffer << '\n'; buffer << std::string(indent + 2, ' ') << "PROJECTION\n"; getProjection().dumpTreeImpl(buffer, format_state, indent + 4); @@ -161,6 +160,13 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); + if (constant_value && rhs_typed.constant_value && *constant_value != *rhs_typed.constant_value) + return false; + else if (constant_value && !rhs_typed.constant_value) + return false; + else if (!constant_value && rhs_typed.constant_value) + return false; + return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name && @@ -178,6 +184,17 @@ void QueryNode::updateTreeHashImpl(HashState & state) const state.update(is_distinct); state.update(is_limit_with_ties); + + if (constant_value) + { + auto constant_dump = applyVisitor(FieldVisitorToString(), constant_value->getValue()); + state.update(constant_dump.size()); + state.update(constant_dump); + + auto constant_value_type_name = constant_value->getType()->getName(); + state.update(constant_value_type_name.size()); + state.update(constant_value_type_name); + } } ASTPtr QueryNode::toASTImpl() const @@ -191,7 +208,7 @@ ASTPtr QueryNode::toASTImpl() const select_query->setExpression(ASTSelectQuery::Expression::SELECT, children[projection_child_index]->toAST()); ASTPtr tables_in_select_query_ast = std::make_shared(); - addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, getJoinTree()); + addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, getJoinTree()); select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select_query_ast)); if (getPrewhere()) @@ -243,6 +260,8 @@ QueryTreeNodePtr QueryNode::cloneImpl() const result_query_node->is_distinct = is_distinct; result_query_node->is_limit_with_ties = is_limit_with_ties; result_query_node->cte_name = cte_name; + result_query_node->projection_columns = projection_columns; + result_query_node->constant_value = constant_value; return result_query_node; } diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 8405517c246..c4ae4270afd 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -1,5 +1,8 @@ #pragma once +#include +#include + #include #include #include @@ -232,8 +235,15 @@ public: return children[offset_child_index]; } - /// Compute query node columns using projection section - NamesAndTypesList computeProjectionColumns() const; + const NamesAndTypes & getProjectionColumns() const + { + return projection_columns; + } + + void resolveProjectionColumns(NamesAndTypes projection_columns_value) + { + projection_columns = std::move(projection_columns_value); + } QueryTreeNodeType getNodeType() const override { @@ -242,6 +252,25 @@ public: String getName() const override; + DataTypePtr getResultType() const override + { + if (constant_value) + return constant_value->getType(); + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method getResultType is not supported for non scalar query node"); + } + + /// Perform constant folding for scalar subquery node + void performConstantFolding(ConstantValuePtr constant_folded_value) + { + constant_value = std::move(constant_folded_value); + } + + ConstantValuePtr getConstantValueOrNull() const override + { + return constant_value; + } + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; bool isEqualImpl(const IQueryTreeNode & rhs) const override; @@ -259,6 +288,8 @@ private: bool is_distinct = false; bool is_limit_with_ties = false; std::string cte_name; + NamesAndTypes projection_columns; + ConstantValuePtr constant_value; static constexpr size_t with_child_index = 0; static constexpr size_t projection_child_index = 1; diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index ea456f3e5fe..0da311b800e 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -117,7 +117,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & } -SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const DataTypePtr & value_type, const Field & value, const Settings & settings) +SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, const Settings & settings) { DataTypes set_element_types = {expression_type}; const auto * lhs_tuple_type = typeid_cast(expression_type.get()); @@ -155,9 +155,7 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const DataTy if (rhs_which_type.isArray()) result_block = createBlockFromCollection(value.get(), set_element_types, tranform_null_in); else if (rhs_which_type.isTuple()) - { result_block = createBlockFromCollection(value.get(), set_element_types, tranform_null_in); - } else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Unsupported type at the right-side of IN. Expected Array or Tuple. Actual {}", diff --git a/src/Analyzer/SetUtils.h b/src/Analyzer/SetUtils.h index e39e388a1fb..7afc8e5259c 100644 --- a/src/Analyzer/SetUtils.h +++ b/src/Analyzer/SetUtils.h @@ -19,12 +19,12 @@ using SetPtr = std::shared_ptr; * Example: SELECT id FROM test_table WHERE id IN ((1, 2), (3, 4)); * * @param expression_type - type of first argument of function IN. - * @param value_type - type of second argument of function IN. * @param value - constant value of second argument of function IN. + * @param value_type - type of second argument of function IN. * @param settings - query settings. * * @return SetPtr for constant value. */ -SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const DataTypePtr & value_type, const Field & value, const Settings & settings); +SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, const Settings & settings); } diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 5af59f50aae..8fbd3bb193e 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -1,6 +1,7 @@ #include #include +#include #include @@ -37,7 +38,7 @@ UnionNode::UnionNode() children[queries_child_index] = std::make_shared(); } -NamesAndTypesList UnionNode::computeProjectionColumns() const +NamesAndTypes UnionNode::computeProjectionColumns() const { std::vector projections; const auto & query_nodes = getQueries().getNodes(); @@ -48,7 +49,7 @@ NamesAndTypesList UnionNode::computeProjectionColumns() const { if (auto * query_node_typed = query_node->as()) { - auto projection_columns = query_node_typed->computeProjectionColumns(); + auto projection_columns = query_node_typed->getProjectionColumns(); query_node_projection = NamesAndTypes(projection_columns.begin(), projection_columns.end()); } else if (auto * union_node_typed = query_node->as()) @@ -63,7 +64,7 @@ NamesAndTypesList UnionNode::computeProjectionColumns() const throw Exception(ErrorCodes::TYPE_MISMATCH, "UNION different number of columns in queries"); } - NamesAndTypesList result_columns; + NamesAndTypes result_columns; size_t projections_size = projections.size(); DataTypes projection_column_types; @@ -120,6 +121,12 @@ void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s if (!cte_name.empty()) buffer << ", cte_name: " << cte_name; + if (constant_value) + { + buffer << ", constant_value: " << constant_value->getValue().dump(); + buffer << ", constant_value_type: " << constant_value->getType()->getName(); + } + buffer << ", union_mode: "; if (union_mode == SelectUnionMode::ALL || union_mode == SelectUnionMode::DISTINCT) @@ -151,6 +158,13 @@ void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s bool UnionNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); + if (constant_value && rhs_typed.constant_value && *constant_value != *rhs_typed.constant_value) + return false; + else if (constant_value && !rhs_typed.constant_value) + return false; + else if (!constant_value && rhs_typed.constant_value) + return false; + return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name; } @@ -161,6 +175,17 @@ void UnionNode::updateTreeHashImpl(HashState & state) const state.update(cte_name.size()); state.update(cte_name); + + if (constant_value) + { + auto constant_dump = applyVisitor(FieldVisitorToString(), constant_value->getValue()); + state.update(constant_dump.size()); + state.update(constant_dump); + + auto constant_value_type_name = constant_value->getType()->getName(); + state.update(constant_value_type_name.size()); + state.update(constant_value_type_name); + } } ASTPtr UnionNode::toASTImpl() const @@ -182,10 +207,10 @@ QueryTreeNodePtr UnionNode::cloneImpl() const result_query_node->is_subquery = is_subquery; result_query_node->is_cte = is_cte; result_query_node->cte_name = cte_name; - result_query_node->union_mode = union_mode; result_query_node->union_modes = union_modes; result_query_node->union_modes_set = union_modes_set; + result_query_node->constant_value = constant_value; return result_query_node; } diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h index 94dbe30f470..f153fda575b 100644 --- a/src/Analyzer/UnionNode.h +++ b/src/Analyzer/UnionNode.h @@ -1,5 +1,8 @@ #pragma once +#include +#include + #include #include #include @@ -91,7 +94,7 @@ public: } /// Compute union projection - NamesAndTypesList computeProjectionColumns() const; + NamesAndTypes computeProjectionColumns() const; QueryTreeNodeType getNodeType() const override { @@ -100,6 +103,25 @@ public: String getName() const override; + DataTypePtr getResultType() const override + { + if (constant_value) + return constant_value->getType(); + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method getResultType is not supported for non scalar union node"); + } + + /// Perform constant folding for scalar union node + void performConstantFolding(ConstantValuePtr constant_folded_value) + { + constant_value = std::move(constant_folded_value); + } + + ConstantValuePtr getConstantValueOrNull() const override + { + return constant_value; + } + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; bool isEqualImpl(const IQueryTreeNode & rhs) const override; @@ -118,6 +140,7 @@ private: SelectUnionMode union_mode; SelectUnionModes union_modes; SelectUnionModesSet union_modes_set; + ConstantValuePtr constant_value; static constexpr size_t queries_child_index = 0; static constexpr size_t children_size = queries_child_index + 1; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 03d7f350db0..e8d3852cf76 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -6,6 +6,10 @@ #include #include +#include +#include +#include +#include namespace DB { @@ -82,7 +86,7 @@ static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expre return result_table_expression; } -void addTableExpressionIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression) +void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression) { auto table_expression_node_type = table_expression->getNodeType(); @@ -126,4 +130,56 @@ void addTableExpressionIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_a } } +QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node) +{ + QueryTreeNodes result; + + std::deque nodes_to_process; + nodes_to_process.push_back(join_tree_node); + + while (!nodes_to_process.empty()) + { + auto node_to_process = std::move(nodes_to_process.front()); + nodes_to_process.pop_front(); + + auto node_type = node_to_process->getNodeType(); + + switch (node_type) + { + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::UNION: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + result.push_back(std::move(node_to_process)); + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + auto & array_join_node = node_to_process->as(); + nodes_to_process.push_back(array_join_node.getTableExpression()); + break; + } + case QueryTreeNodeType::JOIN: + { + auto & join_node = node_to_process->as(); + nodes_to_process.push_back(join_node.getLeftTableExpression()); + nodes_to_process.push_back(join_node.getRightTableExpression()); + break; + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected node type for table expression. Expected table, table function, query, union, join or array join. Actual {}", + node_to_process->getNodeTypeName()); + } + } + } + + return result; +} + } diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 19d5a20f994..7ee0bdfcaea 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -14,6 +14,9 @@ bool isNameOfInFunction(const std::string & function_name); /** Add table expression in tables in select query children. * If table expression node is not of identifier node, table node, query node, table function node, join node or array join node type throws logical error exception. */ -void addTableExpressionIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression); +void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression); + +/// Extract table, table function query, union from join tree +QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node); } diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index e0be9ba6109..60894d331a1 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -62,7 +62,7 @@ Block InterpreterDescribeQuery::getSampleBlock(bool include_subcolumns) BlockIO InterpreterDescribeQuery::execute() { - ColumnsDescription columns; + std::vector columns; StorageSnapshotPtr storage_snapshot; const auto & ast = query_ptr->as(); @@ -85,12 +85,20 @@ BlockIO InterpreterDescribeQuery::execute() names_and_types = InterpreterSelectWithUnionQuery::getSampleBlock(select_query, current_context).getNamesAndTypesList(); } - columns = ColumnsDescription(std::move(names_and_types)); + for (auto && [name, type] : names_and_types) + { + ColumnDescription description; + description.name = std::move(name); + description.type = std::move(type); + columns.emplace_back(std::move(description)); + } } else if (table_expression.table_function) { TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression.table_function, getContext()); - columns = table_function_ptr->getActualTableStructure(getContext()); + auto table_function_column_descriptions = table_function_ptr->getActualTableStructure(getContext()); + for (const auto & table_function_column_description : table_function_column_descriptions) + columns.emplace_back(table_function_column_description); } else { @@ -101,7 +109,9 @@ BlockIO InterpreterDescribeQuery::execute() auto metadata_snapshot = table->getInMemoryMetadataPtr(); storage_snapshot = table->getStorageSnapshot(metadata_snapshot, getContext()); - columns = metadata_snapshot->getColumns(); + auto metadata_column_descriptions = metadata_snapshot->getColumns(); + for (const auto & metadata_column_description : metadata_column_descriptions) + columns.emplace_back(metadata_column_description); } bool extend_object_types = settings.describe_extend_object_types && storage_snapshot; diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 94f1739fe3c..73dde2a4aa8 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -115,6 +115,18 @@ InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( { } +InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( + const QueryTreeNodePtr & query_tree_, + const SelectQueryOptions & select_query_options_, + ContextPtr context_) + : WithContext(context_) + , query(query_tree_->toAST()) + , query_tree(query_tree_) + , select_query_options(select_query_options_) + , planner(query_tree, select_query_options, context_) +{ +} + Block InterpreterSelectQueryAnalyzer::getSampleBlock() { planner.buildQueryPlanIfNeeded(); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 2f4b472537b..3e3e2c18e54 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -20,6 +20,11 @@ public: const SelectQueryOptions & select_query_options_, ContextPtr context_); + /// Initialize interpreter with query tree + InterpreterSelectQueryAnalyzer(const QueryTreeNodePtr & query_tree_, + const SelectQueryOptions & select_query_options_, + ContextPtr context_); + Block getSampleBlock(); QueryPlan && extractQueryPlan() &&; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index bb1a1c2b145..5fd06572c5c 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -191,7 +191,7 @@ public: static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { - return child_node->getNodeType() != QueryTreeNodeType::QUERY; + return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; } }; @@ -218,7 +218,7 @@ AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodePtr & quer assertNoAggregatesFunctions(aggregate_function_node_child, "inside another aggregate function", planner_context); auto & aggregagte_function_node_typed = aggregate_function_node->as(); - String node_name = calculateActionsDAGNodeName(aggregate_function_node.get(), data.planner_context, node_to_name); + String node_name = calculateActionNodeName(aggregate_function_node, data.planner_context, node_to_name); auto [_, inserted] = unique_aggregate_action_node_names.emplace(node_name); if (!inserted) continue; @@ -231,8 +231,8 @@ AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodePtr & quer for (const auto & parameter_node : parameters_nodes) { - const auto & constant_node = parameter_node->as(); - aggregate_description.parameters.push_back(constant_node.getConstantValue()); + /// Function parameters constness validated during analysis stage + aggregate_description.parameters.push_back(parameter_node->getConstantValue().getValue()); } const auto & arguments_nodes = aggregagte_function_node_typed.getArguments().getNodes(); @@ -240,7 +240,7 @@ AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodePtr & quer for (const auto & argument_node : arguments_nodes) { - String argument_node_name = calculateActionsDAGNodeName(argument_node.get(), data.planner_context, node_to_name); + String argument_node_name = calculateActionNodeName(argument_node, data.planner_context, node_to_name); aggregate_description.argument_names.emplace_back(std::move(argument_node_name)); } @@ -274,7 +274,7 @@ public: if (!column_node) return; - String column_node_name = calculateActionsDAGNodeName(node.get(), data.planner_context); + String column_node_name = calculateActionNodeName(node, data.planner_context); if (!data.group_by_keys_column_names.contains(column_node_name)) throw Exception(ErrorCodes::NOT_AN_AGGREGATE, "Column {} is not under aggregate function and not in GROUP BY keys", @@ -287,65 +287,12 @@ public: if (function_node && function_node->isAggregateFunction()) return false; - return child_node->getNodeType() != QueryTreeNodeType::QUERY; + return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; } }; using ValidateGroupByColumnsVisitor = typename ValidateGroupByColumnsMatcher::Visitor; -class CollectTableExpressionIdentifiersVisitor -{ -public: - void visit(const QueryTreeNodePtr & join_tree_node, PlannerContext & planner_context) - { - auto & table_expression_node_to_identifier = planner_context.getTableExpressionNodeToIdentifier(); - auto join_tree_node_type = join_tree_node->getNodeType(); - - switch (join_tree_node_type) - { - case QueryTreeNodeType::QUERY: - [[fallthrough]]; - case QueryTreeNodeType::UNION: - [[fallthrough]]; - case QueryTreeNodeType::TABLE: - [[fallthrough]]; - case QueryTreeNodeType::TABLE_FUNCTION: - { - std::string table_expression_identifier = std::to_string(table_expression_node_to_identifier.size()); - table_expression_node_to_identifier.emplace(join_tree_node.get(), table_expression_identifier); - break; - } - case QueryTreeNodeType::JOIN: - { - auto & join_node = join_tree_node->as(); - visit(join_node.getLeftTableExpression(), planner_context); - - std::string table_expression_identifier = std::to_string(table_expression_node_to_identifier.size()); - table_expression_node_to_identifier.emplace(join_tree_node.get(), table_expression_identifier); - - visit(join_node.getRightTableExpression(), planner_context); - break; - } - case QueryTreeNodeType::ARRAY_JOIN: - { - auto & array_join_node = join_tree_node->as(); - visit(array_join_node.getTableExpression(), planner_context); - - std::string table_expression_identifier = std::to_string(table_expression_node_to_identifier.size()); - table_expression_node_to_identifier.emplace(join_tree_node.get(), table_expression_identifier); - break; - } - default: - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected query, union, table, table function, join or array join query node. Actual {} {}", - join_tree_node->getNodeTypeName(), - join_tree_node->formatASTForErrorMessage()); - } - } - } -}; - class CollectSourceColumnsMatcher { public: @@ -375,7 +322,7 @@ public: auto & table_expression_node_to_columns = data.planner_context.getTableExpressionNodeToColumns(); - auto [it, _] = table_expression_node_to_columns.emplace(column_source_node.get(), TableExpressionColumns()); + auto [it, _] = table_expression_node_to_columns.emplace(column_source_node, TableExpressionColumns()); auto & table_expression_columns = it->second; if (column_node->hasExpression()) @@ -399,20 +346,20 @@ public: if (!column_already_exists) { - auto column_identifier = data.planner_context.getColumnUniqueIdentifier(column_source_node.get(), column_node->getColumnName()); - data.planner_context.registerColumnNode(column_node, column_identifier); + auto column_identifier = data.planner_context.getColumnUniqueIdentifier(column_source_node, column_node->getColumnName()); + data.planner_context.registerColumnNode(node, column_identifier); table_expression_columns.addColumn(column_node->getColumn(), column_identifier); } else { auto column_identifier = table_expression_columns.getColumnIdentifierOrThrow(column_node->getColumnName()); - data.planner_context.registerColumnNode(column_node, column_identifier); + data.planner_context.registerColumnNode(node, column_identifier); } } static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { - return child_node->getNodeType() != QueryTreeNodeType::QUERY; + return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; } }; @@ -453,7 +400,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, /** Use default columns to support case when there are no columns in query. * Example: SELECT 1; */ - const auto & [it, _] = table_expression_node_to_columns.emplace(table_expression.get(), TableExpressionColumns()); + const auto & [it, _] = table_expression_node_to_columns.emplace(table_expression, TableExpressionColumns()); auto & table_expression_columns = it->second; if (table_node || table_function_node) @@ -467,8 +414,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, std::optional read_additional_column; - bool plan_has_multiple_table_expressions = table_expression_node_to_columns.size() > 1; - if (column_names.empty() && (plan_has_multiple_table_expressions || storage->getName() == "SystemOne")) + if (column_names.empty()) { auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); read_additional_column = column_names_and_types.front(); @@ -476,7 +422,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, if (read_additional_column) { - auto column_identifier = planner_context->getColumnUniqueIdentifier(table_expression.get(), read_additional_column->name); + auto column_identifier = planner_context->getColumnUniqueIdentifier(table_expression, read_additional_column->name); column_names.push_back(read_additional_column->name); table_expression_columns.addColumn(*read_additional_column, column_identifier); } @@ -589,13 +535,13 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, const auto & join_node_using_column_node_type = join_node_using_column_node.getColumnType(); if (!left_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) { - auto left_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(left_inner_column_node.get()); + auto left_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(left_inner_column_node); left_plan_column_name_to_cast_type.emplace(left_inner_column_identifier, join_node_using_column_node_type); } if (!right_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) { - auto right_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(right_inner_column_node.get()); + auto right_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(right_inner_column_node); right_plan_column_name_to_cast_type.emplace(right_inner_column_identifier, join_node_using_column_node_type); } } @@ -754,8 +700,8 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, auto & using_join_left_join_column_node = using_join_columns_list.getNodes().at(0); auto & using_join_right_join_column_node = using_join_columns_list.getNodes().at(1); - auto left_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_left_join_column_node.get()); - auto right_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_right_join_column_node.get()); + auto left_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_left_join_column_node); + auto right_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_right_join_column_node); table_join_clause.key_names_left.push_back(left_column_identifier); table_join_clause.key_names_right.push_back(right_column_identifier); @@ -1057,9 +1003,6 @@ void Planner::buildQueryPlanIfNeeded() storage_limits.push_back(getStorageLimits(*current_context, select_query_options)); select_query_info.storage_limits = std::make_shared(storage_limits); - CollectTableExpressionIdentifiersVisitor collect_table_expression_identifiers_visitor; - collect_table_expression_identifiers_visitor.visit(query_node.getJoinTree(), *planner_context); - CollectSourceColumnsVisitor::Data data {*planner_context}; CollectSourceColumnsVisitor collect_source_columns_visitor(data); collect_source_columns_visitor.visit(query_tree); @@ -1158,37 +1101,32 @@ void Planner::buildQueryPlanIfNeeded() const auto & projection_action_dag_nodes = projection_actions->getOutputs(); size_t projection_action_dag_nodes_size = projection_action_dag_nodes.size(); - auto & projection_nodes = query_node.getProjection().getNodes(); - size_t projection_nodes_size = projection_nodes.size(); + auto projection_columns = query_node.getProjectionColumns(); + size_t projection_columns_size = projection_columns.size(); - if (projection_nodes_size != projection_action_dag_nodes_size) + if (projection_columns_size != projection_action_dag_nodes_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "QueryTree projection nodes size mismatch. Expected {}. Actual {}", projection_action_dag_nodes_size, - projection_nodes_size); + projection_columns_size); - Names projection_names; - projection_names.reserve(projection_nodes_size); + Names projection_action_names; + projection_action_names.reserve(projection_columns_size); - NamesWithAliases projection_names_with_aliases; - projection_names_with_aliases.reserve(projection_nodes_size); + NamesWithAliases projection_action_names_with_display_aliases; + projection_action_names_with_display_aliases.reserve(projection_columns_size); - for (size_t i = 0; i < projection_nodes_size; ++i) + for (size_t i = 0; i < projection_columns_size; ++i) { - auto & node = projection_nodes[i]; - auto node_name = node->getName(); + auto & projection_column = projection_columns[i]; const auto * action_dag_node = projection_action_dag_nodes[i]; const auto & actions_dag_node_name = action_dag_node->result_name; - projection_names.push_back(actions_dag_node_name); - - if (node->hasAlias()) - projection_names_with_aliases.push_back({actions_dag_node_name, node->getAlias()}); - else - projection_names_with_aliases.push_back({actions_dag_node_name, node_name}); + projection_action_names.push_back(actions_dag_node_name); + projection_action_names_with_display_aliases.push_back({actions_dag_node_name, projection_column.name}); } - projection_actions->project(projection_names_with_aliases); + projection_actions->project(projection_action_names_with_display_aliases); actions_chain.addStep(std::make_unique(std::move(projection_actions))); size_t projection_action_step_index = actions_chain.getLastStepIndex(); @@ -1269,7 +1207,7 @@ void Planner::buildQueryPlanIfNeeded() if (table_expression_node_to_columns.size() == 1) { auto it = table_expression_node_to_columns.begin(); - const auto * table_expression_node = it->first; + const auto & table_expression_node = it->first; if (const auto * table_node = table_expression_node->as()) storage_has_evenly_distributed_read = table_node->getStorage()->hasEvenlyDistributedRead(); else if (const auto * table_function_node = table_expression_node->as()) @@ -1312,7 +1250,7 @@ void Planner::buildQueryPlanIfNeeded() query_plan.getCurrentDataStream(), limits, limit_hint_for_distinct, - projection_names, + projection_action_names, pre_distinct, settings.optimize_distinct_in_order); @@ -1379,7 +1317,7 @@ void Planner::buildQueryPlanIfNeeded() if (query_node.hasOffset()) { /// Validated during query analysis stage - limit_offset = query_node.getOffset()->as().getConstantValue().safeGet(); + limit_offset = query_node.getOffset()->getConstantValue().getValue().safeGet(); } if (query_node.hasLimit()) @@ -1389,7 +1327,7 @@ void Planner::buildQueryPlanIfNeeded() bool limit_with_ties = query_node.isLimitWithTies(); /// Validated during query analysis stage - UInt64 limit_length = query_node.getLimit()->as().getConstantValue().safeGet(); + UInt64 limit_length = query_node.getLimit()->getConstantValue().getValue().safeGet(); SortDescription limit_with_ties_sort_description; diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index ce29c2df38b..ded4b30204e 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -6,9 +6,13 @@ #include #include #include +#include +#include #include +#include + #include #include @@ -35,6 +39,12 @@ namespace ErrorCodes namespace { +String calculateConstantActionNodeName(const Field & constant_literal, const DataTypePtr & constant_type) +{ + auto constant_name = applyVisitor(FieldVisitorToString(), constant_literal); + return "__constant_" + constant_name + "_" + constant_type->getName(); +} + class ActionsScopeNode { public: @@ -157,6 +167,8 @@ private: NodeNameAndNodeMinLevel visitColumn(const QueryTreeNodePtr & node); + NodeNameAndNodeMinLevel visitConstantValue(const Field & constant_literal, const DataTypePtr & constant_type); + NodeNameAndNodeMinLevel visitConstant(const QueryTreeNodePtr & node); NodeNameAndNodeMinLevel visitLambda(const QueryTreeNodePtr & node); @@ -165,8 +177,10 @@ private: NodeNameAndNodeMinLevel visitFunction(const QueryTreeNodePtr & node); + NodeNameAndNodeMinLevel visitQueryOrUnion(const QueryTreeNodePtr & node); + std::vector actions_stack; - std::unordered_map node_to_node_name; + std::unordered_map node_to_node_name; const PlannerContextPtr planner_context; }; @@ -199,12 +213,16 @@ ActionsDAG::NodeRawConstPtrs PlannerActionsVisitorImpl::visit(QueryTreeNodePtr e PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitImpl(QueryTreeNodePtr node) { - if (auto * column_node = node->as()) + auto node_type = node->getNodeType(); + + if (node_type == QueryTreeNodeType::COLUMN) return visitColumn(node); - else if (auto * constant_node = node->as()) + else if (node_type == QueryTreeNodeType::CONSTANT) return visitConstant(node); - else if (auto * function_node = node->as()) + else if (node_type == QueryTreeNodeType::FUNCTION) return visitFunction(node); + else if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) + return visitQueryOrUnion(node); throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected only column, constant or function node. Actual {}", @@ -213,7 +231,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitColumn(const QueryTreeNodePtr & node) { - auto column_node_name = calculateActionsDAGNodeName(node.get(), *planner_context, node_to_node_name); + auto column_node_name = calculateActionNodeName(node, *planner_context, node_to_node_name); const auto & column_node = node->as(); Int64 actions_stack_size = static_cast(actions_stack.size() - 1); @@ -231,16 +249,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi return {column_node_name, 0}; } -PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitConstant(const QueryTreeNodePtr & node) +PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitConstantValue(const Field & constant_literal, const DataTypePtr & constant_type) { - auto constant_node_name = calculateActionsDAGNodeName(node.get(), *planner_context, node_to_node_name); - const auto & constant_node = node->as(); - const auto & literal = constant_node.getConstantValue(); + auto constant_node_name = calculateConstantActionNodeName(constant_literal, constant_type); ColumnWithTypeAndName column; column.name = constant_node_name; - column.type = constant_node.getResultType(); - column.column = column.type->createColumnConst(1, literal); + column.type = constant_type; + column.column = column.type->createColumnConst(1, constant_literal); actions_stack[0].addConstantIfNecessary(constant_node_name, column); @@ -254,6 +270,12 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi return {constant_node_name, 0}; } +PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitConstant(const QueryTreeNodePtr & node) +{ + const auto & constant_node = node->as(); + return visitConstantValue(constant_node.getValue(), constant_node.getResultType()); +} + PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitLambda(const QueryTreeNodePtr & node) { auto & lambda_node = node->as(); @@ -302,7 +324,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi } } - auto lambda_node_name = calculateActionsDAGNodeName(node.get(), *planner_context); + auto lambda_node_name = calculateActionNodeName(node, *planner_context); auto function_capture = std::make_shared( lambda_actions, captured_column_names, lambda_arguments_names_and_types, result_type, lambda_expression_node_name); actions_stack.pop_back(); @@ -329,7 +351,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma auto in_second_argument = function_node.getArguments().getNodes().at(1); const auto & global_planner_context = planner_context->getGlobalPlannerContext(); - auto set_key = global_planner_context->getSetKey(in_second_argument.get()); + auto set_key = global_planner_context->getSetKey(in_second_argument); auto prepared_set = global_planner_context->getSetOrThrow(set_key); auto column_set = ColumnSet::create(1, std::move(prepared_set)); @@ -349,7 +371,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma actions_stack_node.addInputConstantColumnIfNecessary(set_key, column); } - node_to_node_name.emplace(in_second_argument.get(), set_key); + node_to_node_name.emplace(in_second_argument, set_key); return {set_key, 0}; } @@ -357,6 +379,9 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitFunction(const QueryTreeNodePtr & node) { const auto & function_node = node->as(); + if (const auto constant_value_or_null = function_node.getConstantValueOrNull()) + return visitConstantValue(constant_value_or_null->getValue(), constant_value_or_null->getType()); + std::optional in_function_second_argument_node_name_with_level; if (function_node.getFunctionName() == "grouping") @@ -410,7 +435,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi level = std::max(level, node_min_level); } - auto function_node_name = calculateActionsDAGNodeName(node.get(), *planner_context, node_to_node_name); + auto function_node_name = calculateActionNodeName(node, *planner_context, node_to_node_name); if (function_node.isAggregateFunction()) { @@ -455,6 +480,16 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi return {function_node_name, level}; } +PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitQueryOrUnion(const QueryTreeNodePtr & node) +{ + const auto constant_value = node->getConstantValueOrNull(); + if (!constant_value) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Scalar subqueries must be evaluated as constants"); + + return visitConstantValue(constant_value->getValue(), constant_value->getType()); +} + } PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_context_) @@ -467,7 +502,7 @@ ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAGPtr actions_ return actions_visitor_impl.visit(expression_node); } -String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name) +String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name) { auto it = node_to_name.find(node); if (it != node_to_name.end()) @@ -488,7 +523,7 @@ String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerCon case QueryTreeNodeType::CONSTANT: { const auto & constant_node = node->as(); - result = "__constant_" + constant_node.getName() + "_" + constant_node.getResultType()->getName(); + result = calculateConstantActionNodeName(constant_node.getValue(), constant_node.getResultType()); break; } case QueryTreeNodeType::FUNCTION: @@ -508,7 +543,7 @@ String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerCon for (size_t i = 0; i < function_parameters_nodes_size; ++i) { const auto & function_parameter_node = function_parameters_nodes[i]; - calculateActionsDAGNodeName(function_parameter_node.get(), planner_context, node_to_name); + buffer << calculateActionNodeName(function_parameter_node, planner_context, node_to_name); if (i + 1 != function_parameters_nodes_size) buffer << ", "; @@ -525,7 +560,7 @@ String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerCon for (size_t i = 0; i < function_arguments_nodes_size; ++i) { const auto & function_argument_node = function_arguments_nodes[i]; - buffer << calculateActionsDAGNodeName(function_argument_node.get(), planner_context, node_to_name); + buffer << calculateActionNodeName(function_argument_node, planner_context, node_to_name); if (i + 1 != function_arguments_nodes_size) buffer << ", "; @@ -565,10 +600,10 @@ String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerCon return result; } -String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerContext & planner_context) +String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerContext & planner_context) { QueryTreeNodeToName empty_map; - return calculateActionsDAGNodeName(node, planner_context, empty_map); + return calculateActionNodeName(node, planner_context, empty_map); } } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index a03a632e2ef..746839b4dc8 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -50,14 +50,14 @@ private: * * For column node column node identifier from planner context is used. */ -using QueryTreeNodeToName = std::unordered_map; -String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name); +using QueryTreeNodeToName = std::unordered_map; +String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name); /** Calculate query tree expression node name action dag name. * * For column node column node identifier from planner context is used. */ -String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerContext & planner_context); +String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerContext & planner_context); } diff --git a/src/Planner/PlannerCollectSets.cpp b/src/Planner/PlannerCollectSets.cpp index a9f0313a0ed..377575f3a80 100644 --- a/src/Planner/PlannerCollectSets.cpp +++ b/src/Planner/PlannerCollectSets.cpp @@ -41,7 +41,7 @@ public: const auto & global_planner_context = planner_context.getGlobalPlannerContext(); const auto & settings = planner_context.getQueryContext()->getSettingsRef(); - String set_key = global_planner_context->getSetKey(in_second_argument.get()); + String set_key = global_planner_context->getSetKey(in_second_argument); auto prepared_set = global_planner_context->getSetOrNull(set_key); if (prepared_set) @@ -55,6 +55,16 @@ public: { global_planner_context->registerSet(set_key, storage_set->getSet()); } + else if (auto constant_value = in_second_argument->getConstantValueOrNull()) + { + auto set = makeSetForConstantValue( + in_first_argument->getResultType(), + constant_value->getValue(), + constant_value->getType(), + settings); + + global_planner_context->registerSet(set_key, std::move(set)); + } else if (in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION) { @@ -66,18 +76,6 @@ public: global_planner_context->registerSet(set_key, set); global_planner_context->registerSubqueryNodeForSet(set_key, SubqueryNodeForSet{in_second_argument, set}); } - else if (in_second_argument_node_type == QueryTreeNodeType::CONSTANT) - { - auto & in_second_argument_constant_node = in_second_argument->as(); - - auto set = makeSetForConstantValue( - in_first_argument->getResultType(), - in_second_argument_constant_node.getResultType(), - in_second_argument_constant_node.getConstantValue(), - settings); - - global_planner_context->registerSet(set_key, std::move(set)); - } else { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, @@ -87,7 +85,8 @@ public: static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { - return child_node->getNodeType() != QueryTreeNodeType::QUERY; + return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || + child_node->getNodeType() == QueryTreeNodeType::UNION); } }; diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index a227af83b9c..1391184e5f3 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -10,7 +10,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -GlobalPlannerContext::SetKey GlobalPlannerContext::getSetKey(const IQueryTreeNode * set_source_node) const +GlobalPlannerContext::SetKey GlobalPlannerContext::getSetKey(const QueryTreeNodePtr & set_source_node) const { auto set_source_hash = set_source_node->getTreeHash(); return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); @@ -61,41 +61,17 @@ PlannerContext::PlannerContext(ContextPtr query_context_, GlobalPlannerContextPt , global_planner_context(std::move(global_planner_context_)) {} -ColumnIdentifier PlannerContext::getColumnUniqueIdentifier(const IQueryTreeNode * column_source_node, std::string column_name) +ColumnIdentifier PlannerContext::getColumnUniqueIdentifier(const QueryTreeNodePtr & column_source_node, std::string column_name) { auto column_unique_prefix = "__column_" + std::to_string(column_identifier_counter); ++column_identifier_counter; - std::string table_expression_identifier; - auto table_expression_identifier_it = table_expression_node_to_identifier.find(column_source_node); - if (table_expression_identifier_it != table_expression_node_to_identifier.end()) - table_expression_identifier = table_expression_identifier_it->second; - std::string debug_identifier_suffix; if (column_source_node->hasAlias()) - { debug_identifier_suffix += column_source_node->getAlias(); - } else if (const auto * table_source_node = column_source_node->as()) - { debug_identifier_suffix += table_source_node->getStorageID().getFullNameNotQuoted(); - } - else - { - auto column_source_node_type = column_source_node->getNodeType(); - if (column_source_node_type == QueryTreeNodeType::JOIN) - debug_identifier_suffix += "join"; - else if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN) - debug_identifier_suffix += "array_join"; - else if (column_source_node_type == QueryTreeNodeType::TABLE_FUNCTION) - debug_identifier_suffix += "table_function"; - else if (column_source_node_type == QueryTreeNodeType::QUERY) - debug_identifier_suffix += "subquery"; - - if (!table_expression_identifier.empty()) - debug_identifier_suffix += '_' + table_expression_identifier; - } if (!column_name.empty()) debug_identifier_suffix += '.' + column_name; @@ -106,13 +82,13 @@ ColumnIdentifier PlannerContext::getColumnUniqueIdentifier(const IQueryTreeNode return column_unique_prefix; } -void PlannerContext::registerColumnNode(const IQueryTreeNode * column_node, const ColumnIdentifier & column_identifier) +void PlannerContext::registerColumnNode(const QueryTreeNodePtr & column_node, const ColumnIdentifier & column_identifier) { assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); column_node_to_column_identifier.emplace(column_node, column_identifier); } -const ColumnIdentifier & PlannerContext::getColumnNodeIdentifierOrThrow(const IQueryTreeNode * column_node) const +const ColumnIdentifier & PlannerContext::getColumnNodeIdentifierOrThrow(const QueryTreeNodePtr & column_node) const { assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); @@ -125,7 +101,7 @@ const ColumnIdentifier & PlannerContext::getColumnNodeIdentifierOrThrow(const IQ return it->second; } -const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const IQueryTreeNode * column_node) const +const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const QueryTreeNodePtr & column_node) const { assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index c57027248cd..95af4858eb2 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -132,7 +132,7 @@ public: using SetKeyToSubqueryNode = std::unordered_map; /// Get set key for query node - SetKey getSetKey(const IQueryTreeNode * set_source_node) const; + SetKey getSetKey(const QueryTreeNodePtr & set_source_node) const; /// Register set for set key void registerSet(const SetKey & key, SetPtr set); @@ -181,33 +181,23 @@ public: return global_planner_context; } - const std::unordered_map & getTableExpressionNodeToIdentifier() const - { - return table_expression_node_to_identifier; - } - - std::unordered_map & getTableExpressionNodeToIdentifier() - { - return table_expression_node_to_identifier; - } - - const std::unordered_map & getTableExpressionNodeToColumns() const + const std::unordered_map & getTableExpressionNodeToColumns() const { return table_expression_node_to_columns; } - std::unordered_map & getTableExpressionNodeToColumns() + std::unordered_map & getTableExpressionNodeToColumns() { return table_expression_node_to_columns; } - ColumnIdentifier getColumnUniqueIdentifier(const IQueryTreeNode * column_source_node, std::string column_name = {}); + ColumnIdentifier getColumnUniqueIdentifier(const QueryTreeNodePtr & column_source_node, std::string column_name = {}); - void registerColumnNode(const IQueryTreeNode * column_node, const ColumnIdentifier & column_identifier); + void registerColumnNode(const QueryTreeNodePtr & column_node, const ColumnIdentifier & column_identifier); - const ColumnIdentifier & getColumnNodeIdentifierOrThrow(const IQueryTreeNode * column_node) const; + const ColumnIdentifier & getColumnNodeIdentifierOrThrow(const QueryTreeNodePtr & column_node) const; - const ColumnIdentifier * getColumnNodeIdentifierOrNull(const IQueryTreeNode * column_node) const; + const ColumnIdentifier * getColumnNodeIdentifierOrNull(const QueryTreeNodePtr & column_node) const; private: /// Query context @@ -217,13 +207,10 @@ private: GlobalPlannerContextPtr global_planner_context; /// Column node to column identifier - std::unordered_map column_node_to_column_identifier; - - /// Table expression to identifier - std::unordered_map table_expression_node_to_identifier; + std::unordered_map column_node_to_column_identifier; /// Table expression node to columns - std::unordered_map table_expression_node_to_columns; + std::unordered_map table_expression_node_to_columns; size_t column_identifier_counter = 0; }; diff --git a/src/Planner/PlannerSorting.cpp b/src/Planner/PlannerSorting.cpp index 869d1611500..301734e11ec 100644 --- a/src/Planner/PlannerSorting.cpp +++ b/src/Planner/PlannerSorting.cpp @@ -24,11 +24,11 @@ namespace std::pair extractWithFillValue(const QueryTreeNodePtr & node) { - const auto & constant_node = node->as(); + auto constant_value = node->getConstantValue(); std::pair result; - result.first = constant_node.getConstantValue(); - result.second = constant_node.getResultType(); + result.first = constant_value.getValue(); + result.second = constant_value.getType(); if (!isColumnedAsNumber(result.second)) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL expression must be constant with numeric type"); @@ -38,15 +38,16 @@ std::pair extractWithFillValue(const QueryTreeNodePtr & node std::pair> extractWithFillStepValue(const QueryTreeNodePtr & node) { - const auto & constant_node = node->as(); - const auto & constant_node_result_type = constant_node.getResultType(); + auto constant_value = node->getConstantValue(); + + const auto & constant_node_result_type = constant_value.getType(); if (const auto * type_interval = typeid_cast(constant_node_result_type.get())) - return std::make_pair(constant_node.getConstantValue(), type_interval->getKind()); + return std::make_pair(constant_value.getValue(), type_interval->getKind()); if (!isColumnedAsNumber(constant_node_result_type)) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL expression must be constant with numeric type"); - return {constant_node.getConstantValue(), {}}; + return {constant_value.getValue(), {}}; } FillColumnDescription extractWithFillDescription(const SortColumnNode & sort_column_node) @@ -125,7 +126,7 @@ SortDescription extractSortDescription(const QueryTreeNodePtr & order_by_node, c { auto & sort_column_node_typed = sort_column_node->as(); - auto column_name = calculateActionsDAGNodeName(sort_column_node_typed.getExpression().get(), planner_context); + auto column_name = calculateActionNodeName(sort_column_node_typed.getExpression(), planner_context); std::shared_ptr collator = sort_column_node_typed.getCollator(); int direction = sort_column_node_typed.getSortDirection() == SortDirection::ASCENDING ? 1 : -1; int nulls_direction = direction; diff --git a/tests/queries/0_stateless/02338_analyzer_constants_basic.reference b/tests/queries/0_stateless/02338_analyzer_constants_basic.reference index 32f8a5eb124..f3a69e4d835 100644 --- a/tests/queries/0_stateless/02338_analyzer_constants_basic.reference +++ b/tests/queries/0_stateless/02338_analyzer_constants_basic.reference @@ -25,11 +25,11 @@ NULL Nullable(Nothing) (1, 1) Tuple(UInt8, UInt8) (1,1) -- -[(1, 1)] Array(Tuple(UInt8, UInt8)) +array((1, 1)) Array(Tuple(UInt8, UInt8)) [(1,1)] NULL Nullable(Nothing) 1 UInt8 \'test\' String [1, 2, 3] Array(UInt8) -[(1, 1), (1, 1)] Array(Tuple(UInt8, UInt8)) +array((1, 1), (1, 1)) Array(Tuple(UInt8, UInt8)) \N 1 test [1,2,3] [(1,1),(1,1)] diff --git a/tests/queries/0_stateless/02339_analyzer_matcher_basic.reference b/tests/queries/0_stateless/02339_analyzer_matcher_basic.reference index 001fe62d4f9..3ca0d303793 100644 --- a/tests/queries/0_stateless/02339_analyzer_matcher_basic.reference +++ b/tests/queries/0_stateless/02339_analyzer_matcher_basic.reference @@ -84,11 +84,15 @@ value String -- 5 UInt8 6 UInt8 +5 6 -- 5 UInt8 6 UInt8 +5 6 Combine EXCEPT, REPLACE, APPLY transformers -- -\'6\' String +toString(6) String +6 -- -\'6\' String +toString(6) String +6 diff --git a/tests/queries/0_stateless/02340_analyzer_functions.reference b/tests/queries/0_stateless/02340_analyzer_functions.reference index 29dcb813557..fe086c69e91 100644 --- a/tests/queries/0_stateless/02340_analyzer_functions.reference +++ b/tests/queries/0_stateless/02340_analyzer_functions.reference @@ -1,4 +1,4 @@ -2 UInt16 +plus(1, 1) UInt16 2 -- plus(dummy, dummy) UInt16 diff --git a/tests/queries/0_stateless/02342_analyzer_compound_types.reference b/tests/queries/0_stateless/02342_analyzer_compound_types.reference index d8e8237f8f5..439ff7d9f39 100644 --- a/tests/queries/0_stateless/02342_analyzer_compound_types.reference +++ b/tests/queries/0_stateless/02342_analyzer_compound_types.reference @@ -20,26 +20,26 @@ value.value_1_level_0 String ('value_0_level_1','value_1_level_1') value_1_level_0 -- alias_value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) -tupleElement(value, \'value_0_level_0\') Tuple(value_0_level_1 String, value_1_level_1 String) -tupleElement(value, \'value_1_level_0\') String +alias_value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value.value_1_level_0 String (('value_0_level_1','value_1_level_1'),'value_1_level_0') ('value_0_level_1','value_1_level_1') value_1_level_0 -- alias_value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) -tupleElement(value, \'value_0_level_0\') Tuple(value_0_level_1 String, value_1_level_1 String) -tupleElement(value, \'value_1_level_0\') String +alias_value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +alias_value.value_1_level_0 String (('value_0_level_1','value_1_level_1'),'value_1_level_0') ('value_0_level_1','value_1_level_1') value_1_level_0 -- alias_value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) -toString(tupleElement(value, \'value_0_level_0\')) String -toString(tupleElement(value, \'value_1_level_0\')) String +toString(alias_value.value_0_level_0) String +toString(alias_value.value_1_level_0) String (('value_0_level_1','value_1_level_1'),'value_1_level_0') (\'value_0_level_1\',\'value_1_level_1\') value_1_level_0 -- -tupleElement(value, \'value_0_level_0\') Tuple(value_0_level_1 String, value_1_level_1 String) -tupleElement(value, \'value_1_level_0\') String +value.value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String) +value.value_1_level_0 String ('value_0_level_1','value_1_level_1') value_1_level_0 -- -toString(tupleElement(value, \'value_0_level_0\')) String -toString(tupleElement(value, \'value_1_level_0\')) String +toString(value.value_0_level_0) String +toString(value.value_1_level_0) String (\'value_0_level_1\',\'value_1_level_1\') value_1_level_0 -- value.value_0_level_0.value_0_level_1 String @@ -47,26 +47,26 @@ value.value_0_level_0.value_1_level_1 String value_0_level_1 value_1_level_1 -- alias_value Tuple(value_0_level_1 String, value_1_level_1 String) -tupleElement(value.value_0_level_0, \'value_0_level_1\') String -tupleElement(value.value_0_level_0, \'value_1_level_1\') String +alias_value.value_0_level_1 String +alias_value.value_1_level_1 String ('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 -- alias_value Tuple(value_0_level_1 String, value_1_level_1 String) -tupleElement(value.value_0_level_0, \'value_0_level_1\') String -tupleElement(value.value_0_level_0, \'value_1_level_1\') String +alias_value.value_0_level_1 String +alias_value.value_1_level_1 String ('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 -- alias_value Tuple(value_0_level_1 String, value_1_level_1 String) -toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String -toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String +toString(alias_value.value_0_level_1) String +toString(alias_value.value_1_level_1) String ('value_0_level_1','value_1_level_1') value_0_level_1 value_1_level_1 -- -tupleElement(value.value_0_level_0, \'value_0_level_1\') String -tupleElement(value.value_0_level_0, \'value_1_level_1\') String +value.value_0_level_0.value_0_level_1 String +value.value_0_level_0.value_1_level_1 String value_0_level_1 value_1_level_1 -- -toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String -toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String +toString(value.value_0_level_0.value_0_level_1) String +toString(value.value_0_level_0.value_1_level_1) String value_0_level_1 value_1_level_1 Array of tuples id UInt64 @@ -83,26 +83,26 @@ value.value_0_level_0.value_1_level_1 Array(String) ['value_0_level_1'] ['value_1_level_1'] -- alias_value Array(Tuple(value_0_level_1 String, value_1_level_1 String)) -tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(String) -tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(String) +alias_value.value_0_level_1 Array(String) +alias_value.value_1_level_1 Array(String) [('value_0_level_1','value_1_level_1')] ['value_0_level_1'] ['value_1_level_1'] -- alias_value Array(Tuple(value_0_level_1 String, value_1_level_1 String)) -tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(String) -tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(String) +alias_value.value_0_level_1 Array(String) +alias_value.value_1_level_1 Array(String) [('value_0_level_1','value_1_level_1')] ['value_0_level_1'] ['value_1_level_1'] -- alias_value Array(Tuple(value_0_level_1 String, value_1_level_1 String)) -toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String -toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String +toString(alias_value.value_0_level_1) String +toString(alias_value.value_1_level_1) String [('value_0_level_1','value_1_level_1')] [\'value_0_level_1\'] [\'value_1_level_1\'] -- -tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(String) -tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(String) +value.value_0_level_0.value_0_level_1 Array(String) +value.value_0_level_0.value_1_level_1 Array(String) ['value_0_level_1'] ['value_1_level_1'] -- -toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String -toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String +toString(value.value_0_level_0.value_0_level_1) String +toString(value.value_0_level_0.value_1_level_1) String [\'value_0_level_1\'] [\'value_1_level_1\'] Nested id UInt64 @@ -119,24 +119,24 @@ value.value_0_level_0.value_1_level_1 Array(Array(String)) [['value_0_level_1']] [['value_1_level_1']] -- value_alias Array(Nested(value_0_level_1 String, value_1_level_1 String)) -tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(Array(String)) -tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(Array(String)) +value_alias.value_0_level_1 Array(Array(String)) +value_alias.value_1_level_1 Array(Array(String)) [[('value_0_level_1','value_1_level_1')]] [['value_0_level_1']] [['value_1_level_1']] -- value_alias Array(Nested(value_0_level_1 String, value_1_level_1 String)) -tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(Array(String)) -tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(Array(String)) +value_alias.value_0_level_1 Array(Array(String)) +value_alias.value_1_level_1 Array(Array(String)) [[('value_0_level_1','value_1_level_1')]] [['value_0_level_1']] [['value_1_level_1']] -- value_alias Array(Nested(value_0_level_1 String, value_1_level_1 String)) -toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String -toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String +toString(value_alias.value_0_level_1) String +toString(value_alias.value_1_level_1) String [[('value_0_level_1','value_1_level_1')]] [[\'value_0_level_1\']] [[\'value_1_level_1\']] -- -tupleElement(value.value_0_level_0, \'value_0_level_1\') Array(Array(String)) -tupleElement(value.value_0_level_0, \'value_1_level_1\') Array(Array(String)) +value.value_0_level_0.value_0_level_1 Array(Array(String)) +value.value_0_level_0.value_1_level_1 Array(Array(String)) [['value_0_level_1']] [['value_1_level_1']] -- -toString(tupleElement(value.value_0_level_0, \'value_0_level_1\')) String -toString(tupleElement(value.value_0_level_0, \'value_1_level_1\')) String +toString(value.value_0_level_0.value_0_level_1) String +toString(value.value_0_level_0.value_1_level_1) String [[\'value_0_level_1\']] [[\'value_1_level_1\']] diff --git a/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.reference b/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.reference index af53c9c3c21..4904e950431 100644 --- a/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.reference +++ b/tests/queries/0_stateless/02343_analyzer_column_transformers_strict.reference @@ -1 +1,2 @@ Value +1 2 diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.reference b/tests/queries/0_stateless/02378_analyzer_projection_names.reference new file mode 100644 index 00000000000..144386d2cc8 --- /dev/null +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.reference @@ -0,0 +1,258 @@ +-- { echoOn } + +SELECT 'Constants'; +Constants +DESCRIBE (SELECT 1, 'Value'); +1 UInt8 +\'Value\' String +SELECT '--'; +-- +DESCRIBE (SELECT 1 + 1, concat('Value_1', 'Value_2')); +plus(1, 1) UInt16 +concat(\'Value_1\', \'Value_2\') String +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)')); +CAST(tuple(1, \'Value\'), \'Tuple (id UInt64, value String)\') Tuple(id UInt64, value String) +SELECT 'Columns'; +Columns +DESCRIBE (SELECT test_table.id, test_table.id, id FROM test_table); +id UInt64 +id UInt64 +id UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT * FROM test_table); +id UInt64 +value String +SELECT '--'; +-- +DESCRIBE (SELECT * APPLY toString FROM test_table); +toString(id) String +toString(value) String +SELECT '--'; +-- +DESCRIBE (SELECT * APPLY x -> toString(x) FROM test_table); +toString(id) String +toString(value) String +SELECT '--'; +-- +DESCRIBE (SELECT tuple_value.* FROM test_table_compound); +tuple_value.value_1 UInt64 +tuple_value.value_2 String +SELECT '--'; +-- +DESCRIBE (SELECT tuple_value.* APPLY x -> x FROM test_table_compound); +tuple_value.value_1 UInt64 +tuple_value.value_2 String +SELECT '--'; +-- +DESCRIBE (SELECT tuple_value.* APPLY toString FROM test_table_compound); +toString(tuple_value.value_1) String +toString(tuple_value.value_2) String +SELECT '--'; +-- +DESCRIBE (SELECT tuple_value.* APPLY x -> toString(x) FROM test_table_compound); +toString(tuple_value.value_1) String +toString(tuple_value.value_2) String +SELECT 'Constants with aliases'; +Constants with aliases +DESCRIBE (SELECT 1 AS a, a AS b, b, b AS c, c, 'Value' AS d, d AS e, e AS f); +a UInt8 +b UInt8 +b UInt8 +c UInt8 +c UInt8 +d String +e String +f String +SELECT '--'; +-- +DESCRIBE (SELECT plus(1 AS a, a AS b), plus(b, b), plus(b, b) AS c, concat('Value' AS d, d) AS e, e); +plus(a, b) UInt16 +plus(b, b) UInt16 +c UInt16 +e String +e String +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.id, a.value); +a Tuple(id UInt64, value String) +a.id UInt64 +a.value String +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.*); +a Tuple(id UInt64, value String) +a.id UInt64 +a.value String +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT id); +a Tuple(id UInt64, value String) +a.value String +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT value); +a Tuple(id UInt64, value String) +a.id UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT value APPLY toString); +a Tuple(id UInt64, value String) +toString(a.id) String +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT value APPLY x -> toString(x)); +a Tuple(id UInt64, value String) +toString(a.id) String +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, untuple(a)); +a Tuple(id UInt64, value String) +tupleElement(a, \'id\') UInt64 +tupleElement(a, \'value\') String +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, untuple(a) AS b); +a Tuple(id UInt64, value String) +b.id UInt64 +b.value String +SELECT 'Columns with aliases'; +Columns with aliases +DESCRIBE (SELECT test_table.id AS a, a, test_table.id AS b, b AS c, c FROM test_table); +a UInt64 +a UInt64 +b UInt64 +c UInt64 +c UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT plus(test_table.id AS a, test_table.id), plus(id, id AS b), plus(b, b), plus(test_table.id, test_table.id) FROM test_table); +plus(a, id) UInt64 +plus(id, b) UInt64 +plus(b, b) UInt64 +plus(id, id) UInt64 +SELECT '--Fix'; +--Fix +DESCRIBE (SELECT test_table.* REPLACE id + (id AS id_alias) AS id, id_alias FROM test_table); +plus(id, id) UInt64 +value String +id_alias UInt64 +SELECT 'Lambda'; +Lambda +DESCRIBE (SELECT arrayMap(x -> x + 1, [1,2,3])); +arrayMap(lambda(tuple(x), plus(x, 1)), [1, 2, 3]) Array(UInt16) +SELECT '--'; +-- +DESCRIBE (SELECT 1 AS a, arrayMap(x -> x + a, [1,2,3])); +a UInt8 +arrayMap(lambda(tuple(x), plus(x, a)), [1, 2, 3]) Array(UInt16) +SELECT '--'; +-- +DESCRIBE (SELECT arrayMap(x -> x + test_table.id + test_table.id + id, [1,2,3]) FROM test_table); +arrayMap(lambda(tuple(x), plus(plus(plus(x, id), id), id)), [1, 2, 3]) Array(UInt64) +SELECT '--'; +-- +DESCRIBE (SELECT arrayMap(x -> x + (test_table.id AS first) + (test_table.id AS second) + id, [1,2,3]) FROM test_table); +arrayMap(lambda(tuple(x), plus(plus(plus(x, first), second), id)), [1, 2, 3]) Array(UInt64) +SELECT '--'; +-- +DESCRIBE (SELECT arrayMap(x -> test_table.* EXCEPT value, [1,2,3]) FROM test_table); +arrayMap(lambda(tuple(x), id), [1, 2, 3]) Array(UInt64) +SELECT '--'; +-- +DESCRIBE (SELECT arrayMap(x -> test_table.* EXCEPT value APPLY x -> x, [1,2,3]) FROM test_table); +arrayMap(lambda(tuple(x), id), [1, 2, 3]) Array(UInt64) +SELECT '--'; +-- +DESCRIBE (SELECT arrayMap(x -> test_table.* EXCEPT value APPLY toString, [1,2,3]) FROM test_table); +arrayMap(lambda(tuple(x), toString(id)), [1, 2, 3]) Array(String) +SELECT '--'; +-- +DESCRIBE (SELECT arrayMap(x -> test_table.* EXCEPT value APPLY x -> toString(x), [1,2,3]) FROM test_table); +arrayMap(lambda(tuple(x), toString(id)), [1, 2, 3]) Array(String) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.*, [1,2,3])); +compound_value Tuple(id UInt64) +arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.* APPLY x -> x, [1,2,3])); +compound_value Tuple(id UInt64) +arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.* APPLY toString, [1,2,3])); +compound_value Tuple(id UInt64) +arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.* APPLY x -> toString(x), [1,2,3])); +compound_value Tuple(id UInt64) +arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, arrayMap(x -> untuple(a), [1,2,3]) FROM test_table); +a Tuple(id UInt64) +arrayMap(lambda(tuple(x), tupleElement(a, \'id\')), [1, 2, 3]) Array(UInt64) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value, [1,2,3])); +compound_value Tuple(id UInt64, value String) +arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value APPLY x -> x, [1,2,3])); +compound_value Tuple(id UInt64, value String) +arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value APPLY toString, [1,2,3])); +compound_value Tuple(id UInt64, value String) +arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value APPLY x -> toString(x), [1,2,3])); +compound_value Tuple(id UInt64, value String) +arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, arrayMap(x -> untuple(a), [1,2,3]) FROM test_table); +a Tuple(id UInt64) +arrayMap(lambda(tuple(x), tupleElement(a, \'id\')), [1, 2, 3]) Array(UInt64) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, arrayMap(x -> untuple(a) AS untupled_value, [1,2,3]) FROM test_table); +a Tuple(id UInt64) +arrayMap(untupled_value, [1, 2, 3]) Array(UInt64) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, untuple(a) AS untupled_value, arrayMap(x -> untupled_value, [1,2,3]) FROM test_table); +a Tuple(id UInt64) +untupled_value.id UInt64 +arrayMap(lambda(tuple(x), untupled_value), [1, 2, 3]) Array(UInt64) +SELECT '--'; +-- +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, untuple(a) AS untupled_value, arrayMap(x -> untupled_value AS untupled_value_in_lambda, [1,2,3]) FROM test_table); +a Tuple(id UInt64) +untupled_value.id UInt64 +arrayMap(untupled_value_in_lambda, [1, 2, 3]) Array(UInt64) +SELECT 'Standalone lambda'; +Standalone lambda +DESCRIBE (WITH x -> x + 1 AS test_lambda SELECT test_lambda(1)); +test_lambda(1) UInt16 +SELECT 'Subquery'; +Subquery +DESCRIBE (SELECT (SELECT 1), (SELECT 2), (SELECT 3) AS a, (SELECT 4)); +_subquery_1 Nullable(UInt8) +_subquery_2 Nullable(UInt8) +a Nullable(UInt8) +_subquery_4 Nullable(UInt8) +SELECT '--'; +-- +DESCRIBE (SELECT arrayMap(x -> (SELECT 1), [1,2,3]), arrayMap(x -> (SELECT 2) AS a, [1, 2, 3]), arrayMap(x -> (SELECT 1), [1,2,3])); +arrayMap(lambda(tuple(x), _subquery_1), [1, 2, 3]) Array(Nullable(UInt8)) +arrayMap(a, [1, 2, 3]) Array(Nullable(UInt8)) +arrayMap(lambda(tuple(x), _subquery_3), [1, 2, 3]) Array(Nullable(UInt8)) diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.sql b/tests/queries/0_stateless/02378_analyzer_projection_names.sql new file mode 100644 index 00000000000..24ae0d239db --- /dev/null +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.sql @@ -0,0 +1,215 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +DROP TABLE IF EXISTS test_table_compound; +CREATE TABLE test_table_compound +( + id UInt64, + tuple_value Tuple(value_1 UInt64, value_2 String) +) ENGINE=TinyLog; + +INSERT INTO test_table_compound VALUES (0, tuple(0, 'Value')); + +-- { echoOn } + +SELECT 'Constants'; + +DESCRIBE (SELECT 1, 'Value'); + +SELECT '--'; + +DESCRIBE (SELECT 1 + 1, concat('Value_1', 'Value_2')); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)')); + +SELECT 'Columns'; + +DESCRIBE (SELECT test_table.id, test_table.id, id FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT * FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT * APPLY toString FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT * APPLY x -> toString(x) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT tuple_value.* FROM test_table_compound); + +SELECT '--'; + +DESCRIBE (SELECT tuple_value.* APPLY x -> x FROM test_table_compound); + + +SELECT '--'; + +DESCRIBE (SELECT tuple_value.* APPLY toString FROM test_table_compound); + +SELECT '--'; + +DESCRIBE (SELECT tuple_value.* APPLY x -> toString(x) FROM test_table_compound); + +SELECT 'Constants with aliases'; + +DESCRIBE (SELECT 1 AS a, a AS b, b, b AS c, c, 'Value' AS d, d AS e, e AS f); + +SELECT '--'; + +DESCRIBE (SELECT plus(1 AS a, a AS b), plus(b, b), plus(b, b) AS c, concat('Value' AS d, d) AS e, e); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.id, a.value); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.*); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT id); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT value); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT value APPLY toString); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, a.* EXCEPT value APPLY x -> toString(x)); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, untuple(a)); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS a, untuple(a) AS b); + +SELECT 'Columns with aliases'; + +DESCRIBE (SELECT test_table.id AS a, a, test_table.id AS b, b AS c, c FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT plus(test_table.id AS a, test_table.id), plus(id, id AS b), plus(b, b), plus(test_table.id, test_table.id) FROM test_table); + +SELECT '--Fix'; + +DESCRIBE (SELECT test_table.* REPLACE id + (id AS id_alias) AS id, id_alias FROM test_table); + +SELECT 'Lambda'; + +DESCRIBE (SELECT arrayMap(x -> x + 1, [1,2,3])); + +SELECT '--'; + +DESCRIBE (SELECT 1 AS a, arrayMap(x -> x + a, [1,2,3])); + +SELECT '--'; + +DESCRIBE (SELECT arrayMap(x -> x + test_table.id + test_table.id + id, [1,2,3]) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT arrayMap(x -> x + (test_table.id AS first) + (test_table.id AS second) + id, [1,2,3]) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT arrayMap(x -> test_table.* EXCEPT value, [1,2,3]) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT arrayMap(x -> test_table.* EXCEPT value APPLY x -> x, [1,2,3]) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT arrayMap(x -> test_table.* EXCEPT value APPLY toString, [1,2,3]) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT arrayMap(x -> test_table.* EXCEPT value APPLY x -> toString(x), [1,2,3]) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.*, [1,2,3])); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.* APPLY x -> x, [1,2,3])); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.* APPLY toString, [1,2,3])); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS compound_value, arrayMap(x -> compound_value.* APPLY x -> toString(x), [1,2,3])); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value, [1,2,3])); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value APPLY x -> x, [1,2,3])); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value APPLY toString, [1,2,3])); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value APPLY x -> toString(x), [1,2,3])); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, arrayMap(x -> untuple(a), [1,2,3]) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, arrayMap(x -> untuple(a) AS untupled_value, [1,2,3]) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, untuple(a) AS untupled_value, arrayMap(x -> untupled_value, [1,2,3]) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, untuple(a) AS untupled_value, arrayMap(x -> untupled_value AS untupled_value_in_lambda, [1,2,3]) FROM test_table); + +SELECT 'Standalone lambda'; + +DESCRIBE (WITH x -> x + 1 AS test_lambda SELECT test_lambda(1)); + +SELECT 'Subquery'; + +DESCRIBE (SELECT (SELECT 1), (SELECT 2), (SELECT 3) AS a, (SELECT 4)); + +SELECT '--'; + +DESCRIBE (SELECT arrayMap(x -> (SELECT 1), [1,2,3]), arrayMap(x -> (SELECT 2) AS a, [1, 2, 3]), arrayMap(x -> (SELECT 1), [1,2,3])); + +-- { echoOff } + +DROP TABLE test_table; +DROP TABLE test_table_compound; From ea44b4be5ef7ae9bb0f4adbb9e1cb378e2f4d13b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 1 Sep 2022 12:26:22 +0200 Subject: [PATCH 082/188] Added subquery_depth support --- src/Analyzer/QueryAnalysisPass.cpp | 66 +++++++++++-------- .../02378_analyzer_projection_names.reference | 9 ++- .../02378_analyzer_projection_names.sql | 4 ++ .../02378_analyzer_subquery_depth.reference | 1 + .../02378_analyzer_subquery_depth.sql | 4 ++ 5 files changed, 52 insertions(+), 32 deletions(-) create mode 100644 tests/queries/0_stateless/02378_analyzer_subquery_depth.reference create mode 100644 tests/queries/0_stateless/02378_analyzer_subquery_depth.sql diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 79354bcb32f..cf6b59b4fac 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -93,6 +93,7 @@ namespace ErrorCodes extern const int INVALID_WITH_FILL_EXPRESSION; extern const int INVALID_LIMIT_EXPRESSION; extern const int EMPTY_LIST_OF_COLUMNS_QUERIED; + extern const int TOO_DEEP_SUBQUERIES; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -183,7 +184,6 @@ namespace ErrorCodes * TODO: JOIN support SELF JOIN with MergeTree. JOIN support matchers. * TODO: WINDOW functions * TODO: Table expression modifiers final, sample_size, sample_offset - * TODO: Scalar subqueries subquery depth * TODO: Support function identifier resolve from parent query scope, if lambda in parent scope does not capture any columns. */ @@ -547,6 +547,9 @@ struct IdentifierResolveScope /// Node to projection name std::unordered_map node_to_projection_name; + /// Subquery depth + size_t subquery_depth = 0; + TableExpressionData & getTableExpressionDataOrThrow(QueryTreeNodePtr table_expression_node) { auto it = table_expression_node_to_data.find(table_expression_node); @@ -826,7 +829,7 @@ private: QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); - void evaluateScalarSubquery(QueryTreeNodePtr & query_tree_node); + void evaluateScalarSubquery(QueryTreeNodePtr & query_tree_node, size_t subquery_depth); /// Resolve identifier functions @@ -942,7 +945,7 @@ QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunction(const std } /// Evaluate scalar subquery and perform constant folding. -void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node) +void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node, size_t subquery_depth) { auto * query_node = node->as(); auto * union_node = node->as(); @@ -952,6 +955,10 @@ void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node) node->getNodeTypeName(), node->formatASTForErrorMessage()); + if ((query_node && query_node->hasConstantValue()) || + (union_node && union_node->hasConstantValue())) + return; + auto subquery_context = Context::createCopy(context); Settings subquery_settings = context->getSettings(); subquery_settings.max_result_rows = 1; @@ -965,9 +972,7 @@ void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node) // context->addScalar(it.first, it.second); } - size_t subquery_depth = 0; - auto options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth + 1, true /*is_subqueyr*/); - + auto options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth, true /*is_subquery*/); auto interpreter = std::make_unique(node, options, subquery_context); auto io = interpreter->execute(); @@ -1043,7 +1048,7 @@ void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node) { auto tuple_column = ColumnTuple::create(block.getColumns()); tuple_column->get(0, scalar_value); - scalar_type = std::make_shared(block.getDataTypes()); + scalar_type = std::make_shared(block.getDataTypes(), block.getNames()); } auto constant_value = std::make_shared(std::move(scalar_value), std::move(scalar_type)); @@ -1263,21 +1268,18 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier { resolveFunction(it->second, scope); } - else if (node_type == QueryTreeNodeType::QUERY) + else if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) { IdentifierResolveScope subquery_scope(it->second, &scope /*parent_scope*/); - resolveQuery(it->second, subquery_scope); + subquery_scope.subquery_depth = scope.subquery_depth + 1; + + if (node_type == QueryTreeNodeType::QUERY) + resolveQuery(it->second, subquery_scope); + else if (node_type == QueryTreeNodeType::UNION) + resolveUnion(it->second, subquery_scope); if (identifier_lookup.isExpressionLookup()) - evaluateScalarSubquery(it->second); - } - else if (node_type == QueryTreeNodeType::UNION) - { - IdentifierResolveScope subquery_scope(it->second, &scope /*parent_scope*/); - resolveUnion(it->second, subquery_scope); - - if (identifier_lookup.isExpressionLookup()) - evaluateScalarSubquery(it->second); + evaluateScalarSubquery(it->second, subquery_scope.subquery_depth); } scope.expressions_in_resolve_process_stack.popNode(); @@ -2663,15 +2665,15 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc in_second_argument = std::move(in_second_argument_query_node); } - else if (query_node) + else if (query_node || union_node) { IdentifierResolveScope subquery_scope(in_second_argument, &scope /*parent_scope*/); - resolveQuery(in_second_argument, subquery_scope); - } - else if (union_node) - { - IdentifierResolveScope subquery_scope(in_second_argument, &scope /*parent_scope*/); - resolveUnion(in_second_argument, subquery_scope); + subquery_scope.subquery_depth = scope.subquery_depth + 1; + + if (query_node) + resolveQuery(in_second_argument, subquery_scope); + else if (union_node) + resolveUnion(in_second_argument, subquery_scope); } } @@ -3178,20 +3180,22 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes case QueryTreeNodeType::QUERY: { IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); + subquery_scope.subquery_depth = scope.subquery_depth + 1; resolveQuery(node, subquery_scope); if (!allow_table_expression) - evaluateScalarSubquery(node); + evaluateScalarSubquery(node, subquery_scope.subquery_depth); break; } case QueryTreeNodeType::UNION: { IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); + subquery_scope.subquery_depth = scope.subquery_depth + 1; resolveUnion(node, subquery_scope); if (!allow_table_expression) - evaluateScalarSubquery(node); + evaluateScalarSubquery(node, subquery_scope.subquery_depth); break; } @@ -3893,12 +3897,14 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, case QueryTreeNodeType::QUERY: { IdentifierResolveScope subquery_scope(join_tree_node, &scope); + subquery_scope.subquery_depth = scope.subquery_depth + 1; resolveQuery(join_tree_node, subquery_scope); break; } case QueryTreeNodeType::UNION: { IdentifierResolveScope subquery_scope(join_tree_node, &scope); + subquery_scope.subquery_depth = scope.subquery_depth + 1; resolveUnion(join_tree_node, subquery_scope); break; } @@ -4132,6 +4138,12 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, */ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope) { + const auto & settings = context->getSettingsRef(); + if (settings.max_subquery_depth && scope.subquery_depth > settings.max_subquery_depth) + throw Exception(ErrorCodes::TOO_DEEP_SUBQUERIES, + "Too deep subqueries. Maximum: {}", + settings.max_subquery_depth.toString()); + auto & query_node_typed = query_node->as(); /// Initialize aliases in query node scope diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.reference b/tests/queries/0_stateless/02378_analyzer_projection_names.reference index 144386d2cc8..f37fb030ed4 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.reference +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.reference @@ -194,11 +194,6 @@ compound_value Tuple(id UInt64) arrayMap(lambda(tuple(x), toString(compound_value.id)), [1, 2, 3]) Array(String) SELECT '--'; -- -DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, arrayMap(x -> untuple(a), [1,2,3]) FROM test_table); -a Tuple(id UInt64) -arrayMap(lambda(tuple(x), tupleElement(a, \'id\')), [1, 2, 3]) Array(UInt64) -SELECT '--'; --- DESCRIBE (SELECT cast(tuple(1, 'Value'), 'Tuple (id UInt64, value String)') AS compound_value, arrayMap(x -> compound_value.* EXCEPT value, [1,2,3])); compound_value Tuple(id UInt64, value String) arrayMap(lambda(tuple(x), compound_value.id), [1, 2, 3]) Array(UInt64) @@ -256,3 +251,7 @@ DESCRIBE (SELECT arrayMap(x -> (SELECT 1), [1,2,3]), arrayMap(x -> (SELECT 2) AS arrayMap(lambda(tuple(x), _subquery_1), [1, 2, 3]) Array(Nullable(UInt8)) arrayMap(a, [1, 2, 3]) Array(Nullable(UInt8)) arrayMap(lambda(tuple(x), _subquery_3), [1, 2, 3]) Array(Nullable(UInt8)) +SELECT '--'; +-- +SELECT (SELECT 1 AS a, 2 AS b) AS c, c.a, c.b; +(1,2) 1 2 diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.sql b/tests/queries/0_stateless/02378_analyzer_projection_names.sql index 24ae0d239db..ef01861c374 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.sql +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.sql @@ -209,6 +209,10 @@ SELECT '--'; DESCRIBE (SELECT arrayMap(x -> (SELECT 1), [1,2,3]), arrayMap(x -> (SELECT 2) AS a, [1, 2, 3]), arrayMap(x -> (SELECT 1), [1,2,3])); +SELECT '--'; + +SELECT (SELECT 1 AS a, 2 AS b) AS c, c.a, c.b; + -- { echoOff } DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02378_analyzer_subquery_depth.reference b/tests/queries/0_stateless/02378_analyzer_subquery_depth.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02378_analyzer_subquery_depth.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02378_analyzer_subquery_depth.sql b/tests/queries/0_stateless/02378_analyzer_subquery_depth.sql new file mode 100644 index 00000000000..3fea94e1f1c --- /dev/null +++ b/tests/queries/0_stateless/02378_analyzer_subquery_depth.sql @@ -0,0 +1,4 @@ +SET use_analyzer = 1; + +SELECT (SELECT a FROM (SELECT 1 AS a)) SETTINGS max_subquery_depth = 1; -- { serverError 162 } +SELECT (SELECT a FROM (SELECT 1 AS a)) SETTINGS max_subquery_depth = 2; From c087eff470866ad99929b4303e60e7ff5eab8c78 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 2 Sep 2022 11:55:09 +0200 Subject: [PATCH 083/188] Perform GROUP BY and aggregates validation during analysis stage --- .../CollectAggregateFunctionVisitor.cpp | 87 +++++++++ .../CollectAggregateFunctionVisitor.h | 23 +++ src/Analyzer/QueryAnalysisPass.cpp | 127 ++++++++++++- src/Analyzer/QueryAnalysisPass.h | 9 +- src/Planner/Planner.cpp | 174 +++++------------- 5 files changed, 289 insertions(+), 131 deletions(-) create mode 100644 src/Analyzer/CollectAggregateFunctionVisitor.cpp create mode 100644 src/Analyzer/CollectAggregateFunctionVisitor.h diff --git a/src/Analyzer/CollectAggregateFunctionVisitor.cpp b/src/Analyzer/CollectAggregateFunctionVisitor.cpp new file mode 100644 index 00000000000..cdf6c986601 --- /dev/null +++ b/src/Analyzer/CollectAggregateFunctionVisitor.cpp @@ -0,0 +1,87 @@ +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_AGGREGATION; +} + +namespace +{ + +class CollectAggregateFunctionNodesMatcher +{ +public: + using Visitor = ConstInDepthQueryTreeVisitor; + + struct Data + { + Data() = default; + + String assert_no_aggregates_place_message; + QueryTreeNodes * aggregate_function_nodes = nullptr; + }; + + static void visit(const QueryTreeNodePtr & node, Data & data) + { + auto * function_node = node->as(); + if (!function_node) + return; + + if (!function_node->isAggregateFunction()) + return; + + if (!data.assert_no_aggregates_place_message.empty()) + throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, + "Aggregate function {} is found {} in query", + function_node->getName(), + data.assert_no_aggregates_place_message); + + if (data.aggregate_function_nodes) + data.aggregate_function_nodes->push_back(node); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) + { + return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; + } +}; + +using CollectAggregateFunctionNodesVisitor = CollectAggregateFunctionNodesMatcher::Visitor; + +} + +QueryTreeNodes collectAggregateFunctionNodes(const QueryTreeNodePtr & node) +{ + QueryTreeNodes result; + + CollectAggregateFunctionNodesVisitor::Data data; + data.aggregate_function_nodes = &result; + + CollectAggregateFunctionNodesVisitor(data).visit(node); + + return result; +} + +void collectAggregateFunctionNodes(const QueryTreeNodePtr & node, QueryTreeNodes & result) +{ + CollectAggregateFunctionNodesVisitor::Data data; + data.aggregate_function_nodes = &result; + + CollectAggregateFunctionNodesVisitor(data).visit(node); +} + +void assertNoAggregateFunctionNodes(const QueryTreeNodePtr & node, const String & assert_no_aggregates_place_message) +{ + CollectAggregateFunctionNodesVisitor::Data data; + data.assert_no_aggregates_place_message = assert_no_aggregates_place_message; + + CollectAggregateFunctionNodesVisitor(data).visit(node); +} + +} diff --git a/src/Analyzer/CollectAggregateFunctionVisitor.h b/src/Analyzer/CollectAggregateFunctionVisitor.h new file mode 100644 index 00000000000..df26a8f4fc3 --- /dev/null +++ b/src/Analyzer/CollectAggregateFunctionVisitor.h @@ -0,0 +1,23 @@ +#pragma once + +#include + +namespace DB +{ + +/** Collect aggregate function nodes in node children. + * Do not visit subqueries. + */ +QueryTreeNodes collectAggregateFunctionNodes(const QueryTreeNodePtr & node); + +/** Collect aggregate function nodes in node children and add them into result. + * Do not visit subqueries. + */ +void collectAggregateFunctionNodes(const QueryTreeNodePtr & node, QueryTreeNodes & result); + +/** Assert that there are not aggregate function nodes in node children. + * Do not visit subqueries. + */ +void assertNoAggregateFunctionNodes(const QueryTreeNodePtr & node, const String & assert_no_aggregates_place_message); + +} diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index cf6b59b4fac..3436c8606d2 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -49,6 +49,7 @@ #include #include #include +#include #include @@ -94,6 +95,7 @@ namespace ErrorCodes extern const int INVALID_LIMIT_EXPRESSION; extern const int EMPTY_LIST_OF_COLUMNS_QUERIED; extern const int TOO_DEEP_SUBQUERIES; + extern const int NOT_AN_AGGREGATE; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -4121,20 +4123,91 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, scope.table_expressions_in_resolve_process.erase(join_tree_node.get()); } +class ValidateGroupByColumnsMatcher +{ +public: + using Visitor = ConstInDepthQueryTreeVisitor; + + struct Data + { + const QueryTreeNodes & group_by_keys_nodes; + }; + + static void visit(const QueryTreeNodePtr & node, Data & data) + { + auto query_tree_node_type = node->getNodeType(); + if (query_tree_node_type == QueryTreeNodeType::FUNCTION || + query_tree_node_type == QueryTreeNodeType::CONSTANT || + query_tree_node_type == QueryTreeNodeType::SORT_COLUMN) + return; + + auto * column_node = node->as(); + if (!column_node) + return; + + auto column_node_source = column_node->getColumnSource(); + if (column_node_source->getNodeType() == QueryTreeNodeType::LAMBDA) + return; + + for (const auto & group_by_key_node : data.group_by_keys_nodes) + { + if (node->isEqual(*group_by_key_node)) + return; + } + + std::string column_name; + + if (column_node_source->hasAlias()) + column_name = column_node_source->getAlias(); + else if (auto * table_node = column_node_source->as()) + column_name = table_node->getStorageID().getFullTableName(); + + column_name += '.' + column_node->getColumnName(); + + throw Exception(ErrorCodes::NOT_AN_AGGREGATE, + "Column {} is not under aggregate function and not in GROUP BY", + column_name); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node, Data & data) + { + auto * child_function_node = child_node->as(); + if (child_function_node) + { + if (child_function_node->isAggregateFunction()) + return false; + + for (const auto & group_by_key_node : data.group_by_keys_nodes) + { + if (child_node->isEqual(*group_by_key_node)) + return false; + } + } + + return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; + } +}; + +using ValidateGroupByColumnsVisitor = typename ValidateGroupByColumnsMatcher::Visitor; + /** Resolve query. * This function modifies query node during resolve. It is caller responsibility to clone query node before resolve * if it is needed for later use. * - * lambda_node - query_tree_node that must have QueryNode type. + * query_node - query_tree_node that must have QueryNode type. * scope - query scope. It is caller responsibility to create it. * * Resolve steps: * 1. Initialize query scope with aliases. - * 2. Register CTE subqueries in scope. + * 2. Register CTE subqueries from WITH section in scope and remove them from WITH section. * 3. Resolve FROM section. - * 4. Resolve expressions in query parts. - * 5. Remove WITH section from query. - * 6. Validate nodes with duplicate aliases. + * 4. Resolve projection columns. + * 5. Resolve expressions in other query parts. + * 6. Remove WITH section from query. + * 7. Validate nodes with duplicate aliases. + * 8. Remove aliases from expression and lambda nodes. + * 9. Validate aggregates. + * 10. Resolve query tree node with projection columns. */ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope) { @@ -4336,12 +4409,56 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier node->removeAlias(); } + /// Remove aliases from expression and lambda nodes + for (auto & [_, node] : scope.alias_name_to_expression_node) node->removeAlias(); for (auto & [_, node] : scope.alias_name_to_lambda_node) node->removeAlias(); + /** Validate aggregates + * + * 1. Check that there are no aggregate functions in WHERE. + * 2. Check that there are no aggregate functions in PREWHERE. + * 3. Check that there are no aggregate functions in another aggregate functions. + * 4. Check that there are no columns that are not specified in GROUP BY keys. + */ + if (query_node_typed.hasWhere()) + assertNoAggregateFunctionNodes(query_node_typed.getWhere(), "in WHERE"); + if (query_node_typed.hasPrewhere()) + assertNoAggregateFunctionNodes(query_node_typed.getPrewhere(), "in PREWHERE"); + + QueryTreeNodes aggregate_function_nodes; + collectAggregateFunctionNodes(query_node_typed.getProjectionNode(), aggregate_function_nodes); + collectAggregateFunctionNodes(query_node_typed.getOrderByNode(), aggregate_function_nodes); + + for (auto & aggregate_function_node : aggregate_function_nodes) + { + for (auto & aggregate_function_node_child : aggregate_function_node->getChildren()) + assertNoAggregateFunctionNodes(aggregate_function_node_child, "inside another aggregate function"); + } + + QueryTreeNodes group_by_keys_nodes; + group_by_keys_nodes.reserve(query_node_typed.getGroupBy().getNodes().size()); + + for (auto & node : query_node_typed.getGroupBy().getNodes()) + { + if (node->hasConstantValue()) + continue; + + group_by_keys_nodes.push_back(node); + } + + if (!query_node_typed.getGroupBy().getNodes().empty() || !aggregate_function_nodes.empty()) + { + ValidateGroupByColumnsVisitor::Data validate_group_by_visitor_data {group_by_keys_nodes}; + ValidateGroupByColumnsVisitor validate_group_by_visitor(validate_group_by_visitor_data); + + validate_group_by_visitor.visit(query_node_typed.getProjectionNode()); + validate_group_by_visitor.visit(query_node_typed.getOrderByNode()); + } + query_node_typed.resolveProjectionColumns(std::move(projection_columns)); } diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index 9fc17e237a6..64c2ad4be40 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -29,7 +29,14 @@ namespace DB * 8. For query node projection columns are calculated. Later passes cannot change type, display name of projection column, and cannot add or remove * columns in projection section. * - * 9. Special functions handling: + * 9. Query is validated. Parts that are validated: + * + * Constness of function parameters. + * Constness of LIMIT and OFFSET. + * Only columns that are specified in GROUP BY keys after GROUP BY. + * No aggregate functions in WHERE, PREWHERE and inside another aggregate functions. + * + * 10. Special functions handling: * Function `untuple` is handled properly. * Function `arrayJoin` is handled properly. * diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 5fd06572c5c..3e543e8685c 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -51,6 +51,7 @@ #include #include #include +#include #include #include @@ -150,75 +151,27 @@ StorageLimits getStorageLimits(const Context & context, const SelectQueryOptions return {limits, leaf_limits}; } -class CollectAggregateFunctionNodesMatcher +QueryTreeNodes extractAggregateFunctionNodes(const QueryTreeNodePtr & query_node) { -public: - using Visitor = ConstInDepthQueryTreeVisitor; + const auto & query_node_typed = query_node->as(); - struct Data - { - Data(String assert_no_aggregates_place_message_, const PlannerContext & planner_context_) - : assert_no_aggregates_place_message(std::move(assert_no_aggregates_place_message_)) - , planner_context(planner_context_) - {} + QueryTreeNodes aggregate_function_nodes; + collectAggregateFunctionNodes(query_node_typed.getProjectionNode(), aggregate_function_nodes); + collectAggregateFunctionNodes(query_node_typed.getOrderByNode(), aggregate_function_nodes); - explicit Data(const PlannerContext & planner_context_) - : planner_context(planner_context_) - {} - - String assert_no_aggregates_place_message; - const PlannerContext & planner_context; - QueryTreeNodes aggregate_function_nodes; - }; - - static void visit(const QueryTreeNodePtr & node, Data & data) - { - auto * function_node = node->as(); - if (!function_node) - return; - - if (!function_node->isAggregateFunction()) - return; - - if (!data.assert_no_aggregates_place_message.empty()) - throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, - "Aggregate function {} is found {} in query", - function_node->getName(), - data.assert_no_aggregates_place_message); - - data.aggregate_function_nodes.push_back(node); - } - - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) - { - return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; - } -}; - -using CollectAggregateFunctionNodesVisitor = CollectAggregateFunctionNodesMatcher::Visitor; - -void assertNoAggregatesFunctions(const QueryTreeNodePtr & node, const String & assert_no_aggregates_place_message, const PlannerContext & planner_context) -{ - CollectAggregateFunctionNodesVisitor::Data data(assert_no_aggregates_place_message, planner_context); - CollectAggregateFunctionNodesVisitor(data).visit(node); + return aggregate_function_nodes; } -AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodePtr & query_tree, const PlannerContext & planner_context) +AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodes & aggregate_function_nodes, const PlannerContext & planner_context) { - CollectAggregateFunctionNodesVisitor::Data data(planner_context); - CollectAggregateFunctionNodesVisitor(data).visit(query_tree); - QueryTreeNodeToName node_to_name; NameSet unique_aggregate_action_node_names; AggregateDescriptions aggregate_descriptions; - for (auto & aggregate_function_node : data.aggregate_function_nodes) + for (const auto & aggregate_function_node : aggregate_function_nodes) { - for (auto & aggregate_function_node_child : aggregate_function_node->getChildren()) - assertNoAggregatesFunctions(aggregate_function_node_child, "inside another aggregate function", planner_context); - - auto & aggregagte_function_node_typed = aggregate_function_node->as(); - String node_name = calculateActionNodeName(aggregate_function_node, data.planner_context, node_to_name); + const auto & aggregagte_function_node_typed = aggregate_function_node->as(); + String node_name = calculateActionNodeName(aggregate_function_node, planner_context, node_to_name); auto [_, inserted] = unique_aggregate_action_node_names.emplace(node_name); if (!inserted) continue; @@ -240,7 +193,7 @@ AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodePtr & quer for (const auto & argument_node : arguments_nodes) { - String argument_node_name = calculateActionNodeName(argument_node, data.planner_context, node_to_name); + String argument_node_name = calculateActionNodeName(argument_node, planner_context, node_to_name); aggregate_description.argument_names.emplace_back(std::move(argument_node_name)); } @@ -251,48 +204,6 @@ AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodePtr & quer return aggregate_descriptions; } -class ValidateGroupByColumnsMatcher -{ -public: - using Visitor = ConstInDepthQueryTreeVisitor; - - struct Data - { - Data(const NameSet & group_by_keys_column_names_, const PlannerContext & planner_context_) - : group_by_keys_column_names(group_by_keys_column_names_) - , planner_context(planner_context_) - {} - - const NameSet & group_by_keys_column_names; - const PlannerContext & planner_context; - QueryTreeNodeToName node_to_name; - }; - - static void visit(const QueryTreeNodePtr & node, Data & data) - { - auto * column_node = node->as(); - if (!column_node) - return; - - String column_node_name = calculateActionNodeName(node, data.planner_context); - if (!data.group_by_keys_column_names.contains(column_node_name)) - throw Exception(ErrorCodes::NOT_AN_AGGREGATE, - "Column {} is not under aggregate function and not in GROUP BY keys", - column_node->formatASTForErrorMessage()); - } - - static bool needChildVisit(const QueryTreeNodePtr & parent_node, const QueryTreeNodePtr & child_node) - { - auto * function_node = parent_node->as(); - if (function_node && function_node->isAggregateFunction()) - return false; - - return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; - } -}; - -using ValidateGroupByColumnsVisitor = typename ValidateGroupByColumnsMatcher::Visitor; - class CollectSourceColumnsMatcher { public: @@ -1026,12 +937,8 @@ void Planner::buildQueryPlanIfNeeded() where_action_step_index = actions_chain.getLastStepIndex(); } - if (query_node.hasWhere()) - assertNoAggregatesFunctions(query_node.getWhere(), "in WHERE", *planner_context); - if (query_node.hasPrewhere()) - assertNoAggregatesFunctions(query_node.getWhere(), "in PREWHERE", *planner_context); - - AggregateDescriptions aggregates_descriptions = extractAggregateDescriptions(query_tree, *planner_context); + auto aggregate_function_nodes = extractAggregateFunctionNodes(query_tree); + AggregateDescriptions aggregates_descriptions = extractAggregateDescriptions(aggregate_function_nodes, *planner_context); ColumnsWithTypeAndName aggregates_columns; aggregates_columns.reserve(aggregates_descriptions.size()); for (auto & aggregate_description : aggregates_descriptions) @@ -1041,27 +948,46 @@ void Planner::buildQueryPlanIfNeeded() Names aggregate_keys; std::optional aggregate_step_index; + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & group_by_input = chain_available_output_columns ? *chain_available_output_columns + : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + /// Only aggregation keys, and aggregates are available for next steps after GROUP BY step + ActionsDAGPtr group_by_actions_dag = std::make_shared(group_by_input); + group_by_actions_dag->getOutputs().clear(); + + PlannerActionsVisitor actions_visitor(planner_context); + if (query_node.hasGroupBy()) { - const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & group_by_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, query_node.getGroupByNode()); - /// Only aggregation keys, and aggregates are available for next steps after GROUP BY step - auto group_by_actions = convertExpressionNodeIntoDAG(query_node.getGroupByNode(), group_by_input, planner_context); - - aggregate_keys.reserve(group_by_actions->getOutputs().size()); - for (auto & output : group_by_actions->getOutputs()) + aggregate_keys.reserve(expression_dag_nodes.size()); + for (auto & expression_dag_node : expression_dag_nodes) { - aggregate_keys_set.insert(output->result_name); - aggregate_keys.push_back(output->result_name); + aggregate_keys_set.insert(expression_dag_node->result_name); + aggregate_keys.push_back(expression_dag_node->result_name); + group_by_actions_dag->getOutputs().push_back(expression_dag_node); } + } - ValidateGroupByColumnsVisitor::Data validate_group_by_visitor_data(aggregate_keys_set, *planner_context); - ValidateGroupByColumnsVisitor validate_columns_visitor(validate_group_by_visitor_data); - validate_columns_visitor.visit(query_node.getProjectionNode()); - validate_columns_visitor.visit(query_node.getOrderByNode()); + if (!aggregate_function_nodes.empty()) + { + for (auto & aggregate_function_node : aggregate_function_nodes) + { + auto & aggregate_function_node_typed = aggregate_function_node->as(); + for (const auto & aggregate_function_node_argument : aggregate_function_node_typed.getArguments().getNodes()) + { + auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, aggregate_function_node_argument); + for (auto & expression_dag_node : expression_dag_nodes) + group_by_actions_dag->getOutputs().push_back(expression_dag_node); + } + } + } - auto aggregate_step = std::make_unique(std::move(group_by_actions), ActionsChainStep::AvailableOutputColumnsStrategy::OUTPUT_NODES, aggregates_columns); + if (!group_by_actions_dag->getOutputs().empty()) + { + auto aggregate_step = std::make_unique(std::move(group_by_actions_dag), ActionsChainStep::AvailableOutputColumnsStrategy::OUTPUT_NODES, aggregates_columns); actions_chain.addStep(std::move(aggregate_step)); aggregate_step_index = actions_chain.getLastStepIndex(); } @@ -1069,15 +995,13 @@ void Planner::buildQueryPlanIfNeeded() std::optional before_order_by_step_index; if (query_node.hasOrderBy()) { - const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & order_by_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); ActionsDAGPtr actions_dag = std::make_shared(order_by_input); auto & actions_dag_outputs = actions_dag->getOutputs(); actions_dag_outputs.clear(); - PlannerActionsVisitor actions_visitor(planner_context); - /** We add only sort column sort expression in before ORDER BY actions DAG. * WITH fill expressions must be constant nodes. */ @@ -1094,7 +1018,7 @@ void Planner::buildQueryPlanIfNeeded() before_order_by_step_index = actions_chain.getLastStepIndex(); } - const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); auto projection_actions = convertExpressionNodeIntoDAG(query_node.getProjectionNode(), projection_input, planner_context); From 80b77f4ccc87c46d7149b3ec337c054db4ce9c0c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 2 Sep 2022 12:30:12 +0200 Subject: [PATCH 084/188] Added HAVING support --- src/Analyzer/QueryAnalysisPass.cpp | 14 +++++++++- src/Analyzer/QueryNode.cpp | 36 ++++++++++++++++++++++++++ src/Analyzer/QueryNode.h | 15 +++++++++++ src/Analyzer/QueryTreeBuilder.cpp | 4 +++ src/Planner/Planner.cpp | 41 ++++++++++++++++++++++++++---- 5 files changed, 104 insertions(+), 6 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 3436c8606d2..c653fb3077a 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -4239,6 +4239,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasGroupBy()) visitor.visit(query_node_typed.getGroupByNode()); + if (query_node_typed.hasHaving()) + visitor.visit(query_node_typed.getHaving()); + if (query_node_typed.hasOrderBy()) visitor.visit(query_node_typed.getOrderByNode()); @@ -4312,6 +4315,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasGroupBy()) resolveExpressionNodeList(query_node_typed.getGroupByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + if (query_node_typed.hasHaving()) + resolveExpressionNode(query_node_typed.getHaving(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + if (query_node_typed.hasOrderBy()) resolveSortColumnsNodeList(query_node_typed.getOrderByNode(), scope); @@ -4430,8 +4436,14 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier assertNoAggregateFunctionNodes(query_node_typed.getPrewhere(), "in PREWHERE"); QueryTreeNodes aggregate_function_nodes; + + if (query_node_typed.hasHaving()) + collectAggregateFunctionNodes(query_node_typed.getHaving(), aggregate_function_nodes); + + if (query_node_typed.hasOrderBy()) + collectAggregateFunctionNodes(query_node_typed.getOrderByNode(), aggregate_function_nodes); + collectAggregateFunctionNodes(query_node_typed.getProjectionNode(), aggregate_function_nodes); - collectAggregateFunctionNodes(query_node_typed.getOrderByNode(), aggregate_function_nodes); for (auto & aggregate_function_node : aggregate_function_nodes) { diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 1785f846fd3..e7e71c6421c 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -63,7 +63,34 @@ String QueryNode::getName() const } if (hasGroupBy()) + { + buffer << " GROUP BY "; buffer << getGroupBy().getName(); + } + + if (hasHaving()) + { + buffer << " HAVING "; + buffer << getHaving()->getName(); + } + + if (hasOrderBy()) + { + buffer << " ORDER BY "; + buffer << getOrderByNode()->getName(); + } + + if (hasLimit()) + { + buffer << " LIMIT "; + buffer << getLimit()->getName(); + } + + if (hasOffset()) + { + buffer << " OFFSET "; + buffer << getOffset()->getName(); + } return buffer.str(); } @@ -138,6 +165,12 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s getGroupBy().dumpTreeImpl(buffer, format_state, indent + 4); } + if (hasHaving()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "HAVING\n"; + getHaving()->dumpTreeImpl(buffer, format_state, indent + 4); + } + if (hasOrderBy()) { buffer << '\n' << std::string(indent + 2, ' ') << "ORDER BY\n"; @@ -220,6 +253,9 @@ ASTPtr QueryNode::toASTImpl() const if (hasGroupBy()) select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, getGroupBy().toAST()); + if (hasHaving()) + select_query->setExpression(ASTSelectQuery::Expression::HAVING, getHaving()->toAST()); + if (hasOrderBy()) select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, getOrderBy().toAST()); diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index c4ae4270afd..510c3719527 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -180,6 +180,21 @@ public: return children[group_by_child_index]; } + bool hasHaving() const + { + return getHaving() != nullptr; + } + + const QueryTreeNodePtr & getHaving() const + { + return children[having_child_index]; + } + + QueryTreeNodePtr & getHaving() + { + return children[having_child_index]; + } + bool hasOrderBy() const { return !getOrderBy().getNodes().empty(); diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 9799d6491dd..9540f592e5e 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -220,6 +220,10 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q if (group_by_list) current_query_tree->getGroupByNode() = buildExpressionList(group_by_list); + auto having_expression = select_query_typed.having(); + if (having_expression) + current_query_tree->getHaving() = buildExpression(having_expression); + auto select_order_by_list = select_query_typed.orderBy(); if (select_order_by_list) current_query_tree->getOrderByNode() = buildSortColumnList(select_order_by_list); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 3e543e8685c..fb79382fa59 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -156,8 +156,13 @@ QueryTreeNodes extractAggregateFunctionNodes(const QueryTreeNodePtr & query_node const auto & query_node_typed = query_node->as(); QueryTreeNodes aggregate_function_nodes; + if (query_node_typed.hasHaving()) + collectAggregateFunctionNodes(query_node_typed.getHaving(), aggregate_function_nodes); + + if (query_node_typed.hasOrderBy()) + collectAggregateFunctionNodes(query_node_typed.getOrderByNode(), aggregate_function_nodes); + collectAggregateFunctionNodes(query_node_typed.getProjectionNode(), aggregate_function_nodes); - collectAggregateFunctionNodes(query_node_typed.getOrderByNode(), aggregate_function_nodes); return aggregate_function_nodes; } @@ -924,7 +929,7 @@ void Planner::buildQueryPlanIfNeeded() ActionsChain actions_chain; std::optional where_action_step_index; - std::string where_action_node_name; + std::string where_filter_action_node_name; if (query_node.hasWhere()) { @@ -932,7 +937,7 @@ void Planner::buildQueryPlanIfNeeded() const auto & where_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); auto where_actions = convertExpressionNodeIntoDAG(query_node.getWhere(), where_input, planner_context); - where_action_node_name = where_actions->getOutputs().at(0)->result_name; + where_filter_action_node_name = where_actions->getOutputs().at(0)->result_name; actions_chain.addStep(std::make_unique(std::move(where_actions))); where_action_step_index = actions_chain.getLastStepIndex(); } @@ -992,6 +997,20 @@ void Planner::buildQueryPlanIfNeeded() aggregate_step_index = actions_chain.getLastStepIndex(); } + std::optional having_action_step_index; + std::string having_filter_action_node_name; + + if (query_node.hasHaving()) + { + chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & having_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + auto having_actions = convertExpressionNodeIntoDAG(query_node.getHaving(), having_input, planner_context); + having_filter_action_node_name = having_actions->getOutputs().at(0)->result_name; + actions_chain.addStep(std::make_unique(std::move(having_actions))); + having_action_step_index = actions_chain.getLastStepIndex(); + } + std::optional before_order_by_step_index; if (query_node.hasOrderBy()) { @@ -1066,10 +1085,10 @@ void Planner::buildQueryPlanIfNeeded() if (where_action_step_index) { auto & where_actions_chain_node = actions_chain.at(*where_action_step_index); - bool remove_filter = !where_actions_chain_node->getChildRequiredOutputColumnsNames().contains(where_action_node_name); + bool remove_filter = !where_actions_chain_node->getChildRequiredOutputColumnsNames().contains(where_filter_action_node_name); auto where_step = std::make_unique(query_plan.getCurrentDataStream(), where_actions_chain_node->getActions(), - where_action_node_name, + where_filter_action_node_name, remove_filter); where_step->setStepDescription("WHERE"); query_plan.addStep(std::move(where_step)); @@ -1162,6 +1181,18 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(aggregating_step)); } + if (having_action_step_index) + { + auto & having_actions_chain_node = actions_chain.at(*having_action_step_index); + bool remove_filter = !having_actions_chain_node->getChildRequiredOutputColumnsNames().contains(having_filter_action_node_name); + auto having_step = std::make_unique(query_plan.getCurrentDataStream(), + having_actions_chain_node->getActions(), + having_filter_action_node_name, + remove_filter); + having_step->setStepDescription("HAVING"); + query_plan.addStep(std::move(having_step)); + } + if (query_node.isDistinct()) { const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); From 216250824b7f79d1c6e973a6e7e04897f9d78d9a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 2 Sep 2022 12:49:38 +0200 Subject: [PATCH 085/188] Added extremes support --- src/Planner/Planner.cpp | 68 +++++++++++++++++++++++++---------------- src/Planner/Planner.h | 2 +- 2 files changed, 42 insertions(+), 28 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index fb79382fa59..f31c09b06a1 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -151,6 +152,28 @@ StorageLimits getStorageLimits(const Context & context, const SelectQueryOptions return {limits, leaf_limits}; } +/** There are no limits on the maximum size of the result for the subquery. + * Since the result of the query is not the result of the entire query. + */ +ContextPtr buildSubqueryContext(const ContextPtr & context) +{ + /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. + * Because the result of this query is not the result of the entire query. + * Constraints work instead + * max_rows_in_set, max_bytes_in_set, set_overflow_mode, + * max_rows_in_join, max_bytes_in_join, join_overflow_mode, + * which are checked separately (in the Set, Join objects). + */ + auto subquery_context = Context::createCopy(context); + Settings subquery_settings = context->getSettings(); + subquery_settings.max_result_rows = 0; + subquery_settings.max_result_bytes = 0; + /// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query). + subquery_settings.extremes = false; + subquery_context->setSettings(subquery_settings); + return subquery_context; +} + QueryTreeNodes extractAggregateFunctionNodes(const QueryTreeNodePtr & query_node) { const auto & query_node_typed = query_node->as(); @@ -364,7 +387,8 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, else if (query_node || union_node) { auto subquery_options = select_query_options.subquery(); - Planner subquery_planner(table_expression, subquery_options, planner_context->getQueryContext(), planner_context->getGlobalPlannerContext()); + auto subquery_context = buildSubqueryContext(planner_context->getQueryContext()); + Planner subquery_planner(table_expression, subquery_options, std::move(subquery_context), planner_context->getGlobalPlannerContext()); subquery_planner.buildQueryPlanIfNeeded(); query_plan = std::move(subquery_planner).extractQueryPlan(); } @@ -750,26 +774,12 @@ void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, const SelectQ for (auto [key, subquery_node_for_set] : subquery_node_to_sets) { - /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. - * Because the result of this query is not the result of the entire query. - * Constraints work instead - * max_rows_in_set, max_bytes_in_set, set_overflow_mode, - * max_rows_in_join, max_bytes_in_join, join_overflow_mode, - * which are checked separately (in the Set, Join objects). - */ - auto subquery_context = Context::createCopy(planner_context->getQueryContext()); - Settings subquery_settings = planner_context->getQueryContext()->getSettings(); - subquery_settings.max_result_rows = 0; - subquery_settings.max_result_bytes = 0; - /// The calculation of `extremes` does not make sense and is not necessary (if you do it, then the `extremes` of the subquery can be taken instead of the whole query). - subquery_settings.extremes = false; - subquery_context->setSettings(subquery_settings); - + auto subquery_context = buildSubqueryContext(planner_context->getQueryContext()); auto subquery_options = select_query_options.subquery(); Planner subquery_planner( subquery_node_for_set.subquery_node, subquery_options, - planner_context->getQueryContext(), + std::move(subquery_context), planner_context->getGlobalPlannerContext()); subquery_planner.buildQueryPlanIfNeeded(); @@ -790,8 +800,7 @@ void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, const SelectQ Planner::Planner(const QueryTreeNodePtr & query_tree_, const SelectQueryOptions & select_query_options_, ContextPtr context_) - : WithContext(context_) - , query_tree(query_tree_) + : query_tree(query_tree_) , select_query_options(select_query_options_) , planner_context(std::make_shared(context_, std::make_shared())) { @@ -807,8 +816,7 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_, const SelectQueryOptions & select_query_options_, ContextPtr context_, GlobalPlannerContextPtr global_planner_context_) - : WithContext(context_) - , query_tree(query_tree_) + : query_tree(query_tree_) , select_query_options(select_query_options_) , planner_context(std::make_shared(context_, std::move(global_planner_context_))) { @@ -824,7 +832,7 @@ void Planner::buildQueryPlanIfNeeded() if (query_plan.isInitialized()) return; - auto current_context = getContext(); + auto query_context = planner_context->getQueryContext(); if (auto * union_query_tree = query_tree->as()) { @@ -842,7 +850,7 @@ void Planner::buildQueryPlanIfNeeded() for (auto & query_node : union_query_tree->getQueries().getNodes()) { - Planner query_planner(query_node, select_query_options, current_context); + Planner query_planner(query_node, select_query_options, query_context); query_planner.buildQueryPlanIfNeeded(); auto query_node_plan = std::make_unique(std::move(query_planner).extractQueryPlan()); query_plans_headers.push_back(query_node_plan->getCurrentDataStream().header); @@ -872,7 +880,7 @@ void Planner::buildQueryPlanIfNeeded() query_plans_streams.push_back(query_node_plan->getCurrentDataStream()); } - const auto & settings = current_context->getSettingsRef(); + const auto & settings = query_context->getSettingsRef(); auto max_threads = settings.max_threads; if (union_mode == SelectUnionMode::ALL || union_mode == SelectUnionMode::DISTINCT) @@ -916,7 +924,7 @@ void Planner::buildQueryPlanIfNeeded() select_query_info.query = select_query_info.original_query; StorageLimitsList storage_limits; - storage_limits.push_back(getStorageLimits(*current_context, select_query_options)); + storage_limits.push_back(getStorageLimits(*query_context, select_query_options)); select_query_info.storage_limits = std::make_shared(storage_limits); CollectSourceColumnsVisitor::Data data {*planner_context}; @@ -1262,12 +1270,18 @@ void Planner::buildQueryPlanIfNeeded() if (!fill_description.empty()) { - InterpolateDescriptionPtr interpolate_descr = nullptr; - auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_description), interpolate_descr); + InterpolateDescriptionPtr interpolate_description = nullptr; + auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_description), interpolate_description); query_plan.addStep(std::move(filling_step)); } } + if (query_context->getSettingsRef().extremes) + { + auto extremes_step = std::make_unique(query_plan.getCurrentDataStream()); + query_plan.addStep(std::move(extremes_step)); + } + UInt64 limit_offset = 0; if (query_node.hasOffset()) { diff --git a/src/Planner/Planner.h b/src/Planner/Planner.h index e110bc044f9..e29b347fdfe 100644 --- a/src/Planner/Planner.h +++ b/src/Planner/Planner.h @@ -16,7 +16,7 @@ using GlobalPlannerContextPtr = std::shared_ptr; class PlannerContext; using PlannerContextPtr = std::shared_ptr; -class Planner : public WithContext +class Planner { public: /// Initialize planner with query tree after analysis phase From 63e48bd869a2c9628089c56f59d5f81dc2f74c3d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 2 Sep 2022 16:36:35 +0200 Subject: [PATCH 086/188] Added GROUP BY CUBE, ROLLUP, TOTALS support --- src/Analyzer/QueryAnalysisPass.cpp | 27 ++++++++- src/Analyzer/QueryAnalysisPass.h | 1 + src/Analyzer/QueryNode.cpp | 23 +++++++- src/Analyzer/QueryNode.h | 45 +++++++++++++++ src/Analyzer/QueryTreeBuilder.cpp | 5 +- src/Planner/Planner.cpp | 89 ++++++++++++++++++++++++++---- 6 files changed, 175 insertions(+), 15 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index c653fb3077a..22bb2d94359 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -4429,6 +4429,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier * 2. Check that there are no aggregate functions in PREWHERE. * 3. Check that there are no aggregate functions in another aggregate functions. * 4. Check that there are no columns that are not specified in GROUP BY keys. + * 5. Validate GROUP BY modifiers. */ if (query_node_typed.hasWhere()) assertNoAggregateFunctionNodes(query_node_typed.getWhere(), "in WHERE"); @@ -4462,7 +4463,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier group_by_keys_nodes.push_back(node); } - if (!query_node_typed.getGroupBy().getNodes().empty() || !aggregate_function_nodes.empty()) + bool has_aggregation = !query_node_typed.getGroupBy().getNodes().empty() || !aggregate_function_nodes.empty(); + + if (has_aggregation) { ValidateGroupByColumnsVisitor::Data validate_group_by_visitor_data {group_by_keys_nodes}; ValidateGroupByColumnsVisitor validate_group_by_visitor(validate_group_by_visitor_data); @@ -4471,6 +4474,28 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier validate_group_by_visitor.visit(query_node_typed.getOrderByNode()); } + if (query_node_typed.isGroupByWithGroupingSets()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "GROUPING SETS are not supported"); + + bool is_rollup_or_cube = query_node_typed.isGroupByWithRollup() || query_node_typed.isGroupByWithCube(); + if (!has_aggregation && (query_node_typed.isGroupByWithGroupingSets() || is_rollup_or_cube)) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS, ROLLUP, CUBE or GROUPING SETS are not supported without aggregation"); + + if (query_node_typed.isGroupByWithGroupingSets() && query_node_typed.isGroupByWithTotals()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS and GROUPING SETS are not supported together"); + + if (query_node_typed.isGroupByWithGroupingSets() && is_rollup_or_cube) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "GROUPING SETS are not supported together with ROLLUP and CUBE"); + + if (query_node_typed.isGroupByWithRollup() && (query_node_typed.isGroupByWithGroupingSets() || query_node_typed.isGroupByWithCube())) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ROLLUP is not supported together with GROUPING SETS and CUBE"); + + if (query_node_typed.isGroupByWithCube() && (query_node_typed.isGroupByWithGroupingSets() || query_node_typed.isGroupByWithRollup())) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "CUBE is not supported together with GROUPING SETS and ROLLUP"); + + if (query_node_typed.hasHaving() && query_node_typed.isGroupByWithTotals() && is_rollup_or_cube) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING"); + query_node_typed.resolveProjectionColumns(std::move(projection_columns)); } diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index 64c2ad4be40..f99973ba5d4 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -35,6 +35,7 @@ namespace DB * Constness of LIMIT and OFFSET. * Only columns that are specified in GROUP BY keys after GROUP BY. * No aggregate functions in WHERE, PREWHERE and inside another aggregate functions. + * GROUP BY modifiers CUBE, ROLLUP, GROUPING SETS and WITH TOTALS. * * 10. Special functions handling: * Function `untuple` is handled properly. diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index e7e71c6421c..6b23c34ca4f 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -106,6 +106,10 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << ", is_cte: " << is_cte; buffer << ", is_distinct: " << is_distinct; buffer << ", is_limit_with_ties: " << is_limit_with_ties; + buffer << ", is_group_by_with_totals: " << is_group_by_with_totals; + buffer << ", is_group_by_with_rollup: " << is_group_by_with_rollup; + buffer << ", is_group_by_with_cube: " << is_group_by_with_cube; + buffer << ", is_group_by_with_grouping_sets: " << is_group_by_with_grouping_sets; if (!cte_name.empty()) buffer << ", cte_name: " << cte_name; @@ -204,7 +208,11 @@ bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name && is_distinct == rhs_typed.is_distinct && - is_limit_with_ties == rhs_typed.is_limit_with_ties; + is_limit_with_ties == rhs_typed.is_limit_with_ties && + is_group_by_with_totals == rhs_typed.is_group_by_with_totals && + is_group_by_with_rollup == rhs_typed.is_group_by_with_rollup && + is_group_by_with_cube == rhs_typed.is_group_by_with_cube && + is_group_by_with_grouping_sets == rhs_typed.is_group_by_with_grouping_sets; } void QueryNode::updateTreeHashImpl(HashState & state) const @@ -217,6 +225,10 @@ void QueryNode::updateTreeHashImpl(HashState & state) const state.update(is_distinct); state.update(is_limit_with_ties); + state.update(is_group_by_with_totals); + state.update(is_group_by_with_rollup); + state.update(is_group_by_with_cube); + state.update(is_group_by_with_grouping_sets); if (constant_value) { @@ -234,6 +246,11 @@ ASTPtr QueryNode::toASTImpl() const { auto select_query = std::make_shared(); select_query->distinct = is_distinct; + select_query->limit_with_ties = is_limit_with_ties; + select_query->group_by_with_totals = is_group_by_with_totals; + select_query->group_by_with_rollup = is_group_by_with_rollup; + select_query->group_by_with_cube = is_group_by_with_cube; + select_query->group_by_with_grouping_sets = is_group_by_with_grouping_sets; if (hasWith()) select_query->setExpression(ASTSelectQuery::Expression::WITH, getWith().toAST()); @@ -295,6 +312,10 @@ QueryTreeNodePtr QueryNode::cloneImpl() const result_query_node->is_cte = is_cte; result_query_node->is_distinct = is_distinct; result_query_node->is_limit_with_ties = is_limit_with_ties; + result_query_node->is_group_by_with_totals = is_group_by_with_totals; + result_query_node->is_group_by_with_rollup = is_group_by_with_rollup; + result_query_node->is_group_by_with_cube = is_group_by_with_cube; + result_query_node->is_group_by_with_grouping_sets = is_group_by_with_grouping_sets; result_query_node->cte_name = cte_name; result_query_node->projection_columns = projection_columns; result_query_node->constant_value = constant_value; diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 510c3719527..420356a4a0a 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -70,6 +70,46 @@ public: return is_limit_with_ties; } + void setIsGroupByWithTotals(bool is_group_by_with_totals_value) + { + is_group_by_with_totals = is_group_by_with_totals_value; + } + + bool isGroupByWithTotals() const + { + return is_group_by_with_totals; + } + + void setIsGroupByWithRollup(bool is_group_by_with_rollup_value) + { + is_group_by_with_rollup = is_group_by_with_rollup_value; + } + + bool isGroupByWithRollup() const + { + return is_group_by_with_rollup; + } + + void setIsGroupByWithCube(bool is_group_by_with_cube_value) + { + is_group_by_with_cube = is_group_by_with_cube_value; + } + + bool isGroupByWithCube() const + { + return is_group_by_with_cube; + } + + void setIsGroupByWithGroupingSets(bool is_group_by_with_grouping_sets_value) + { + is_group_by_with_grouping_sets = is_group_by_with_grouping_sets_value; + } + + bool isGroupByWithGroupingSets() const + { + return is_group_by_with_grouping_sets; + } + bool hasWith() const { return !getWith().getNodes().empty(); @@ -302,6 +342,11 @@ private: bool is_cte = false; bool is_distinct = false; bool is_limit_with_ties = false; + bool is_group_by_with_totals = false; + bool is_group_by_with_rollup = false; + bool is_group_by_with_cube = false; + bool is_group_by_with_grouping_sets = false; + std::string cte_name; NamesAndTypes projection_columns; ConstantValuePtr constant_value; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 9540f592e5e..8746973721e 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -196,7 +196,10 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q current_query_tree->setCTEName(cte_name); current_query_tree->setIsDistinct(select_query_typed.distinct); current_query_tree->setIsLimitWithTies(select_query_typed.limit_with_ties); - + current_query_tree->setIsGroupByWithTotals(select_query_typed.group_by_with_totals); + current_query_tree->setIsGroupByWithCube(select_query_typed.group_by_with_cube); + current_query_tree->setIsGroupByWithRollup(select_query_typed.group_by_with_rollup); + current_query_tree->setIsGroupByWithGroupingSets(select_query_typed.group_by_with_grouping_sets); current_query_tree->getJoinTree() = buildJoinTree(select_query_typed.tables()); current_query_tree->setOriginalAST(select_query); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index f31c09b06a1..d15ff4bf294 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -26,6 +26,9 @@ #include #include #include +#include +#include +#include #include #include @@ -82,21 +85,20 @@ namespace ErrorCodes * TODO: JOIN support ASOF. JOIN support strictness. JOIN support constants. JOIN support ON t1.id = t1.id * TODO: JOIN drop unnecessary columns after ON, USING section * TODO: Support display names - * TODO: Support RBAC. Support RBAC for ALIAS columns. + * TODO: Support RBAC. Support RBAC for ALIAS columns * TODO: Support distributed query processing * TODO: Support PREWHERE - * TODO: Support GROUP BY, HAVING - * TODO: Support ORDER BY, LIMIT + * TODO: Support GROUPING SETS, grouping function + * TODO: Support ORDER BY * TODO: Support WINDOW FUNCTIONS * TODO: Support DISTINCT * TODO: Support trivial count optimization - * TODO: Support totals, extremes * TODO: Support projections * TODO: Support read in order optimization * TODO: UNION storage limits * TODO: Interpreter resources * TODO: Support max streams - * TODO: Support GROUPINS SETS, const aggregation keys, overflow row + * TODO: Support GROUPINS SETS, const aggregation keys * TODO: Support interpolate, LIMIT BY. * TODO: Support ORDER BY read in order optimization * TODO: Support GROUP BY read in order optimization @@ -968,6 +970,7 @@ void Planner::buildQueryPlanIfNeeded() /// Only aggregation keys, and aggregates are available for next steps after GROUP BY step ActionsDAGPtr group_by_actions_dag = std::make_shared(group_by_input); group_by_actions_dag->getOutputs().clear(); + std::unordered_set group_by_actions_dag_output_nodes_names; PlannerActionsVisitor actions_visitor(planner_context); @@ -978,9 +981,13 @@ void Planner::buildQueryPlanIfNeeded() aggregate_keys.reserve(expression_dag_nodes.size()); for (auto & expression_dag_node : expression_dag_nodes) { + if (group_by_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) + continue; + aggregate_keys_set.insert(expression_dag_node->result_name); aggregate_keys.push_back(expression_dag_node->result_name); group_by_actions_dag->getOutputs().push_back(expression_dag_node); + group_by_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); } } @@ -993,13 +1000,25 @@ void Planner::buildQueryPlanIfNeeded() { auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, aggregate_function_node_argument); for (auto & expression_dag_node : expression_dag_nodes) + { + if (group_by_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) + continue; + group_by_actions_dag->getOutputs().push_back(expression_dag_node); + group_by_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); + } } } } if (!group_by_actions_dag->getOutputs().empty()) { + /** For non ordinary GROUP BY we add virtual __grouping_set column + * With set number, which is used as an additional key at the stage of merging aggregating data. + */ + if (query_node.isGroupByWithRollup() || query_node.isGroupByWithCube() || query_node.isGroupByWithGroupingSets()) + aggregates_columns.emplace_back(nullptr, std::make_shared(), "__grouping_set"); + auto aggregate_step = std::make_unique(std::move(group_by_actions_dag), ActionsChainStep::AvailableOutputColumnsStrategy::OUTPUT_NODES, aggregates_columns); actions_chain.addStep(std::move(aggregate_step)); aggregate_step_index = actions_chain.getLastStepIndex(); @@ -1102,6 +1121,8 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(where_step)); } + bool having_executed = false; + if (!aggregates_descriptions.empty() || query_node.hasGroupBy()) { if (aggregate_step_index) @@ -1114,8 +1135,6 @@ void Planner::buildQueryPlanIfNeeded() } const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); - - bool overflow_row = false; bool query_analyzer_const_aggregation_keys = false; const auto stats_collecting_params = Aggregator::Params::StatsCollectingParams( @@ -1124,10 +1143,16 @@ void Planner::buildQueryPlanIfNeeded() settings.max_entries_for_hash_table_stats, settings.max_size_to_preallocate_for_aggregation); + bool aggregate_overflow_row = + query_node.isGroupByWithTotals() && + settings.max_rows_to_group_by && + settings.group_by_overflow_mode == OverflowMode::ANY && + settings.totals_mode != TotalsMode::AFTER_HAVING_EXCLUSIVE; + Aggregator::Params aggregator_params = Aggregator::Params( aggregate_keys, aggregates_descriptions, - overflow_row, + aggregate_overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode, settings.group_by_two_level_threshold, @@ -1168,13 +1193,14 @@ void Planner::buildQueryPlanIfNeeded() const bool should_produce_results_in_order_of_bucket_number = select_query_options.to_stage == QueryProcessingStage::WithMergeableState && settings.distributed_aggregation_memory_efficient; - bool aggregate_final = select_query_options.to_stage > QueryProcessingStage::WithMergeableState; - InputOrderInfoPtr input_order_info; + bool aggregate_final = + select_query_options.to_stage > QueryProcessingStage::WithMergeableState && + !query_node.isGroupByWithTotals() && !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube(); auto aggregating_step = std::make_unique( query_plan.getCurrentDataStream(), - std::move(aggregator_params), + aggregator_params, std::move(grouping_sets_params), aggregate_final, settings.max_block_size, @@ -1187,9 +1213,48 @@ void Planner::buildQueryPlanIfNeeded() std::move(group_by_sort_description), should_produce_results_in_order_of_bucket_number); query_plan.addStep(std::move(aggregating_step)); + + if (query_node.isGroupByWithRollup()) + { + auto rollup_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(aggregator_params), true /*final*/, settings.group_by_use_nulls); + query_plan.addStep(std::move(rollup_step)); + } + else if (query_node.isGroupByWithCube()) + { + auto cube_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(aggregator_params), true /*final*/, settings.group_by_use_nulls); + query_plan.addStep(std::move(cube_step)); + } + + if (query_node.isGroupByWithTotals()) + { + bool remove_having_filter = false; + std::shared_ptr having_actions; + + if (having_action_step_index) + { + auto & having_actions_chain_node = actions_chain.at(*having_action_step_index); + remove_having_filter = !having_actions_chain_node->getChildRequiredOutputColumnsNames().contains(having_filter_action_node_name); + having_actions = having_actions_chain_node->getActions(); + having_executed = true; + } + + bool final = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube(); + auto totals_having_step = std::make_unique( + query_plan.getCurrentDataStream(), + aggregates_descriptions, + aggregate_overflow_row, + having_actions, + having_filter_action_node_name, + remove_having_filter, + settings.totals_mode, + settings.totals_auto_threshold, + final); + + query_plan.addStep(std::move(totals_having_step)); + } } - if (having_action_step_index) + if (!having_executed && having_action_step_index) { auto & having_actions_chain_node = actions_chain.at(*having_action_step_index); bool remove_filter = !having_actions_chain_node->getChildRequiredOutputColumnsNames().contains(having_filter_action_node_name); From a5ce4d7823e26319e02710f8e402deab43b679b8 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 2 Sep 2022 17:41:19 +0200 Subject: [PATCH 087/188] Added GROUPING SETS support --- src/Analyzer/QueryAnalysisPass.cpp | 35 +++++++++++--- src/Analyzer/QueryTreeBuilder.cpp | 19 +++++++- src/Planner/Planner.cpp | 78 ++++++++++++++++++++++++------ 3 files changed, 110 insertions(+), 22 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 22bb2d94359..e6fb3fc41cd 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -4313,7 +4313,17 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier resolveExpressionNode(query_node_typed.getWhere(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); if (query_node_typed.hasGroupBy()) - resolveExpressionNodeList(query_node_typed.getGroupByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + { + if (query_node_typed.isGroupByWithGroupingSets()) + { + for (auto & grouping_sets_keys_list_node : query_node_typed.getGroupBy().getNodes()) + resolveExpressionNodeList(grouping_sets_keys_list_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + } + else + { + resolveExpressionNodeList(query_node_typed.getGroupByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + } + } if (query_node_typed.hasHaving()) resolveExpressionNode(query_node_typed.getHaving(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); @@ -4457,10 +4467,24 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier for (auto & node : query_node_typed.getGroupBy().getNodes()) { - if (node->hasConstantValue()) - continue; + if (query_node_typed.isGroupByWithGroupingSets()) + { + auto & grouping_set_keys = node->as(); + for (auto & grouping_set_key : grouping_set_keys.getNodes()) + { + if (grouping_set_key->hasConstantValue()) + continue; - group_by_keys_nodes.push_back(node); + group_by_keys_nodes.push_back(grouping_set_key); + } + } + else + { + if (node->hasConstantValue()) + continue; + + group_by_keys_nodes.push_back(node); + } } bool has_aggregation = !query_node_typed.getGroupBy().getNodes().empty() || !aggregate_function_nodes.empty(); @@ -4474,9 +4498,6 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier validate_group_by_visitor.visit(query_node_typed.getOrderByNode()); } - if (query_node_typed.isGroupByWithGroupingSets()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "GROUPING SETS are not supported"); - bool is_rollup_or_cube = query_node_typed.isGroupByWithRollup() || query_node_typed.isGroupByWithCube(); if (!has_aggregation && (query_node_typed.isGroupByWithGroupingSets() || is_rollup_or_cube)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS, ROLLUP, CUBE or GROUPING SETS are not supported without aggregation"); diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 8746973721e..5f096440885 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -221,7 +221,24 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q auto group_by_list = select_query_typed.groupBy(); if (group_by_list) - current_query_tree->getGroupByNode() = buildExpressionList(group_by_list); + { + auto & group_by_children = group_by_list->children; + + if (current_query_tree->isGroupByWithGroupingSets()) + { + auto grouping_sets_list_node = std::make_shared(); + + for (auto & grouping_sets_keys : group_by_children) + { + auto grouping_sets_keys_list_node = buildExpressionList(grouping_sets_keys); + current_query_tree->getGroupBy().getNodes().emplace_back(std::move(grouping_sets_keys_list_node)); + } + } + else + { + current_query_tree->getGroupByNode() = buildExpressionList(group_by_list); + } + } auto having_expression = select_query_typed.having(); if (having_expression) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d15ff4bf294..b17804162f6 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -88,7 +88,7 @@ namespace ErrorCodes * TODO: Support RBAC. Support RBAC for ALIAS columns * TODO: Support distributed query processing * TODO: Support PREWHERE - * TODO: Support GROUPING SETS, grouping function + * TODO: Support grouping function * TODO: Support ORDER BY * TODO: Support WINDOW FUNCTIONS * TODO: Support DISTINCT @@ -959,7 +959,6 @@ void Planner::buildQueryPlanIfNeeded() for (auto & aggregate_description : aggregates_descriptions) aggregates_columns.emplace_back(nullptr, aggregate_description.function->getReturnType(), aggregate_description.column_name); - NameSet aggregate_keys_set; Names aggregate_keys; std::optional aggregate_step_index; @@ -973,21 +972,73 @@ void Planner::buildQueryPlanIfNeeded() std::unordered_set group_by_actions_dag_output_nodes_names; PlannerActionsVisitor actions_visitor(planner_context); + GroupingSetsParamsList grouping_sets_parameters_list; if (query_node.hasGroupBy()) { - auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, query_node.getGroupByNode()); - - aggregate_keys.reserve(expression_dag_nodes.size()); - for (auto & expression_dag_node : expression_dag_nodes) + if (query_node.isGroupByWithGroupingSets()) { - if (group_by_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) - continue; + for (auto & grouping_set_keys_list_node : query_node.getGroupBy().getNodes()) + { + auto & grouping_set_keys_list_node_typed = grouping_set_keys_list_node->as(); + grouping_sets_parameters_list.emplace_back(); + auto & grouping_sets_parameters = grouping_sets_parameters_list.back(); - aggregate_keys_set.insert(expression_dag_node->result_name); - aggregate_keys.push_back(expression_dag_node->result_name); - group_by_actions_dag->getOutputs().push_back(expression_dag_node); - group_by_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); + for (auto & grouping_set_key_node : grouping_set_keys_list_node_typed.getNodes()) + { + auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, grouping_set_key_node); + aggregate_keys.reserve(expression_dag_nodes.size()); + + for (auto & expression_dag_node : expression_dag_nodes) + { + grouping_sets_parameters.used_keys.push_back(expression_dag_node->result_name); + if (group_by_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) + continue; + + aggregate_keys.push_back(expression_dag_node->result_name); + group_by_actions_dag->getOutputs().push_back(expression_dag_node); + group_by_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); + } + } + } + + for (auto & grouping_sets_parameter : grouping_sets_parameters_list) + { + NameSet grouping_sets_used_keys; + Names grouping_sets_keys; + + for (auto & key : grouping_sets_parameter.used_keys) + { + auto [_, inserted] = grouping_sets_used_keys.insert(key); + if (inserted) + grouping_sets_keys.push_back(key); + } + + for (auto & key : aggregate_keys) + { + if (grouping_sets_used_keys.contains(key)) + continue; + + grouping_sets_parameter.missing_keys.push_back(key); + } + + grouping_sets_parameter.used_keys = std::move(grouping_sets_keys); + } + } + else + { + auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, query_node.getGroupByNode()); + aggregate_keys.reserve(expression_dag_nodes.size()); + + for (auto & expression_dag_node : expression_dag_nodes) + { + if (group_by_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) + continue; + + aggregate_keys.push_back(expression_dag_node->result_name); + group_by_actions_dag->getOutputs().push_back(expression_dag_node); + group_by_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); + } } } @@ -1170,7 +1221,6 @@ void Planner::buildQueryPlanIfNeeded() stats_collecting_params ); - GroupingSetsParamsList grouping_sets_params; SortDescription group_by_sort_description; auto merge_threads = settings.max_threads; @@ -1201,7 +1251,7 @@ void Planner::buildQueryPlanIfNeeded() auto aggregating_step = std::make_unique( query_plan.getCurrentDataStream(), aggregator_params, - std::move(grouping_sets_params), + std::move(grouping_sets_parameters_list), aggregate_final, settings.max_block_size, settings.aggregation_in_order_max_block_bytes, From 504cb2b38f94ac01c2732711f29594d10e00551c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 3 Sep 2022 14:11:09 +0200 Subject: [PATCH 088/188] Added GROUPING function support --- src/Analyzer/ColumnNode.cpp | 9 ++ src/Analyzer/ColumnNode.h | 9 +- src/Analyzer/QueryAnalysisPass.cpp | 114 ++++++++++++++++- src/Analyzer/QueryAnalysisPass.h | 4 +- src/Functions/grouping.h | 21 ++++ src/Planner/Planner.cpp | 20 +-- src/Planner/PlannerAggregation.cpp | 195 +++++++++++++++++++++++++++++ src/Planner/PlannerAggregation.h | 21 ++++ 8 files changed, 371 insertions(+), 22 deletions(-) create mode 100644 src/Planner/PlannerAggregation.cpp create mode 100644 src/Planner/PlannerAggregation.h diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index f62d94bda14..4dd3f85fa99 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -43,6 +43,15 @@ QueryTreeNodePtr ColumnNode::getColumnSource() const return lock; } +QueryTreeNodePtr ColumnNode::getColumnSourceOrNull() const +{ + auto lock = column_source.lock(); + if (!lock) + return nullptr; + + return lock; +} + void ColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t indent) const { buffer << std::string(indent, ' ') << "COLUMN id: " << state.getNodeId(this); diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index 62d53733588..1ef680842d0 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -95,11 +95,10 @@ public: */ QueryTreeNodePtr getColumnSource() const; - /// Get column source weak pointer - QueryTreeNodeWeakPtr getColumnSourceWeak() const - { - return column_source; - } + /** Get column source. + * If column source is not valid null is returned. + */ + QueryTreeNodePtr getColumnSourceOrNull() const; QueryTreeNodeType getNodeType() const override { diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index e6fb3fc41cd..dc7a8994da7 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -67,6 +67,7 @@ #include #include +#include #include @@ -96,6 +97,9 @@ namespace ErrorCodes extern const int EMPTY_LIST_OF_COLUMNS_QUERIED; extern const int TOO_DEEP_SUBQUERIES; extern const int NOT_AN_AGGREGATE; + extern const int ILLEGAL_AGGREGATION; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -2813,7 +2817,18 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc } else if (function_name == "grouping") { - /// It is responsibility of planner to handle grouping function + /// It is responsibility of planner to perform additional handling of grouping function + if (function_arguments_size == 0) + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, + "Function GROUPING expects at least one argument"); + else if (function_arguments_size > 64) + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, + "Function GROUPING can have up to 64 arguments, but {} provided", + function_arguments_size); + + auto grouping_function = std::make_shared(); + auto grouping_function_adaptor = std::make_shared(std::move(grouping_function)); + function_node.resolveAsFunction(std::move(grouping_function_adaptor), std::make_shared()); return; } @@ -3440,7 +3455,6 @@ String QueryAnalyzer::calculateProjectionNodeDisplayName(QueryTreeNodePtr & node if (resolved_identifier_result.resolved_identifier && resolved_identifier_result.isResolvedFromJoinTree()) { projection_name_from_scope = try_to_get_projection_name_from_scope(node); - std::cout << "Projection name from scope " << projection_name_from_scope << std::endl; if (!projection_name_from_scope.empty()) return projection_name_from_scope; @@ -4136,11 +4150,36 @@ public: static void visit(const QueryTreeNodePtr & node, Data & data) { auto query_tree_node_type = node->getNodeType(); - if (query_tree_node_type == QueryTreeNodeType::FUNCTION || - query_tree_node_type == QueryTreeNodeType::CONSTANT || + if (query_tree_node_type == QueryTreeNodeType::CONSTANT || query_tree_node_type == QueryTreeNodeType::SORT_COLUMN) return; + auto * function_node = node->as(); + if (function_node && function_node->getFunctionName() == "grouping") + { + auto & grouping_function_arguments_nodes = function_node->getArguments().getNodes(); + for (auto & grouping_function_arguments_node : grouping_function_arguments_nodes) + { + bool found_argument_in_group_by_keys = false; + + for (const auto & group_by_key_node : data.group_by_keys_nodes) + { + if (grouping_function_arguments_node->isEqual(*group_by_key_node)) + { + found_argument_in_group_by_keys = true; + break; + } + } + + if (!found_argument_in_group_by_keys) + throw Exception(ErrorCodes::NOT_AN_AGGREGATE, + "GROUPING function argument {} is not in GROUP BY", + grouping_function_arguments_node->formatASTForErrorMessage()); + } + + return; + } + auto * column_node = node->as(); if (!column_node) return; @@ -4188,7 +4227,44 @@ public: } }; -using ValidateGroupByColumnsVisitor = typename ValidateGroupByColumnsMatcher::Visitor; +using ValidateGroupByColumnsVisitor = ValidateGroupByColumnsMatcher::Visitor; + +class ValidateGroupingFunctionNodesMatcher +{ +public: + using Visitor = ConstInDepthQueryTreeVisitor; + + struct Data + { + String assert_no_grouping_function_place_message; + }; + + static void visit(const QueryTreeNodePtr & node, Data & data) + { + auto * function_node = node->as(); + if (function_node && function_node->getFunctionName() == "grouping") + throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, + "GROUPING function {} is found {} in query", + function_node->formatASTForErrorMessage(), + data.assert_no_grouping_function_place_message); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) + { + return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; + } +}; + +using ValidateGroupingFunctionNodesVisitor = ValidateGroupingFunctionNodesMatcher::Visitor; + +void assertNoGroupingFunction(const QueryTreeNodePtr & node, const String & assert_no_grouping_function_place_message) +{ + ValidateGroupingFunctionNodesVisitor::Data data; + data.assert_no_grouping_function_place_message = assert_no_grouping_function_place_message; + + ValidateGroupingFunctionNodesVisitor visitor(data); + visitor.visit(node); +} /** Resolve query. * This function modifies query node during resolve. It is caller responsibility to clone query node before resolve @@ -4442,9 +4518,16 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier * 5. Validate GROUP BY modifiers. */ if (query_node_typed.hasWhere()) + { assertNoAggregateFunctionNodes(query_node_typed.getWhere(), "in WHERE"); + assertNoGroupingFunction(query_node_typed.getWhere(), "in WHERE"); + } + if (query_node_typed.hasPrewhere()) + { assertNoAggregateFunctionNodes(query_node_typed.getPrewhere(), "in PREWHERE"); + assertNoGroupingFunction(query_node_typed.getPrewhere(), "in PREWHERE"); + } QueryTreeNodes aggregate_function_nodes; @@ -4459,7 +4542,10 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier for (auto & aggregate_function_node : aggregate_function_nodes) { for (auto & aggregate_function_node_child : aggregate_function_node->getChildren()) + { assertNoAggregateFunctionNodes(aggregate_function_node_child, "inside another aggregate function"); + assertNoGroupingFunction(aggregate_function_node_child, "inside another aggregate function"); + } } QueryTreeNodes group_by_keys_nodes; @@ -4487,6 +4573,17 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier } } + if (query_node_typed.getGroupBy().getNodes().empty()) + { + if (query_node_typed.hasHaving()) + assertNoGroupingFunction(query_node_typed.getHaving(), "in HAVING without GROUP BY"); + + if (query_node_typed.hasOrderBy()) + assertNoGroupingFunction(query_node_typed.getOrderByNode(), "in ORDER BY without GROUP BY"); + + assertNoGroupingFunction(query_node_typed.getProjectionNode(), "in SELECT without GROUP BY"); + } + bool has_aggregation = !query_node_typed.getGroupBy().getNodes().empty() || !aggregate_function_nodes.empty(); if (has_aggregation) @@ -4494,8 +4591,13 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier ValidateGroupByColumnsVisitor::Data validate_group_by_visitor_data {group_by_keys_nodes}; ValidateGroupByColumnsVisitor validate_group_by_visitor(validate_group_by_visitor_data); + if (query_node_typed.hasHaving()) + validate_group_by_visitor.visit(query_node_typed.getHaving()); + + if (query_node_typed.hasOrderBy()) + validate_group_by_visitor.visit(query_node_typed.getOrderByNode()); + validate_group_by_visitor.visit(query_node_typed.getProjectionNode()); - validate_group_by_visitor.visit(query_node_typed.getOrderByNode()); } bool is_rollup_or_cube = query_node_typed.isGroupByWithRollup() || query_node_typed.isGroupByWithCube(); diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index f99973ba5d4..5e098783fd8 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -33,7 +33,9 @@ namespace DB * * Constness of function parameters. * Constness of LIMIT and OFFSET. - * Only columns that are specified in GROUP BY keys after GROUP BY. + * In SELECT, ORDER BY only columns that are specified in GROUP BY keys after GROUP BY are used. + * GROUPING function arguments are specified in GROUP BY keys. + * No GROUPING function if there is no GROUP BY. * No aggregate functions in WHERE, PREWHERE and inside another aggregate functions. * GROUP BY modifiers CUBE, ROLLUP, GROUPING SETS and WITH TOTALS. * diff --git a/src/Functions/grouping.h b/src/Functions/grouping.h index b9ef6ffc107..5c4f5851635 100644 --- a/src/Functions/grouping.h +++ b/src/Functions/grouping.h @@ -13,6 +13,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + class FunctionGroupingBase : public IFunction { protected: @@ -71,6 +76,22 @@ public: } }; +class FunctionGrouping : public FunctionGroupingBase +{ +public: + explicit FunctionGrouping() + : FunctionGroupingBase(ColumnNumbers()) + {} + + String getName() const override { return "grouping"; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t) const override + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Method executeImpl is not supported for 'grouping' function"); + } +}; + class FunctionGroupingOrdinary : public FunctionGroupingBase { public: diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index b17804162f6..1173ae48604 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -2,7 +2,6 @@ #include -#include #include #include @@ -61,6 +60,7 @@ #include #include #include +#include #include #include #include @@ -88,7 +88,6 @@ namespace ErrorCodes * TODO: Support RBAC. Support RBAC for ALIAS columns * TODO: Support distributed query processing * TODO: Support PREWHERE - * TODO: Support grouping function * TODO: Support ORDER BY * TODO: Support WINDOW FUNCTIONS * TODO: Support DISTINCT @@ -959,7 +958,7 @@ void Planner::buildQueryPlanIfNeeded() for (auto & aggregate_description : aggregates_descriptions) aggregates_columns.emplace_back(nullptr, aggregate_description.function->getReturnType(), aggregate_description.column_name); - Names aggregate_keys; + Names aggregation_keys; std::optional aggregate_step_index; const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); @@ -987,7 +986,7 @@ void Planner::buildQueryPlanIfNeeded() for (auto & grouping_set_key_node : grouping_set_keys_list_node_typed.getNodes()) { auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, grouping_set_key_node); - aggregate_keys.reserve(expression_dag_nodes.size()); + aggregation_keys.reserve(expression_dag_nodes.size()); for (auto & expression_dag_node : expression_dag_nodes) { @@ -995,7 +994,7 @@ void Planner::buildQueryPlanIfNeeded() if (group_by_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) continue; - aggregate_keys.push_back(expression_dag_node->result_name); + aggregation_keys.push_back(expression_dag_node->result_name); group_by_actions_dag->getOutputs().push_back(expression_dag_node); group_by_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); } @@ -1014,7 +1013,7 @@ void Planner::buildQueryPlanIfNeeded() grouping_sets_keys.push_back(key); } - for (auto & key : aggregate_keys) + for (auto & key : aggregation_keys) { if (grouping_sets_used_keys.contains(key)) continue; @@ -1028,14 +1027,14 @@ void Planner::buildQueryPlanIfNeeded() else { auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, query_node.getGroupByNode()); - aggregate_keys.reserve(expression_dag_nodes.size()); + aggregation_keys.reserve(expression_dag_nodes.size()); for (auto & expression_dag_node : expression_dag_nodes) { if (group_by_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) continue; - aggregate_keys.push_back(expression_dag_node->result_name); + aggregation_keys.push_back(expression_dag_node->result_name); group_by_actions_dag->getOutputs().push_back(expression_dag_node); group_by_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); } @@ -1070,6 +1069,7 @@ void Planner::buildQueryPlanIfNeeded() if (query_node.isGroupByWithRollup() || query_node.isGroupByWithCube() || query_node.isGroupByWithGroupingSets()) aggregates_columns.emplace_back(nullptr, std::make_shared(), "__grouping_set"); + resolveGroupingFunctions(query_tree, aggregation_keys, grouping_sets_parameters_list, *planner_context); auto aggregate_step = std::make_unique(std::move(group_by_actions_dag), ActionsChainStep::AvailableOutputColumnsStrategy::OUTPUT_NODES, aggregates_columns); actions_chain.addStep(std::move(aggregate_step)); aggregate_step_index = actions_chain.getLastStepIndex(); @@ -1201,7 +1201,7 @@ void Planner::buildQueryPlanIfNeeded() settings.totals_mode != TotalsMode::AFTER_HAVING_EXCLUSIVE; Aggregator::Params aggregator_params = Aggregator::Params( - aggregate_keys, + aggregation_keys, aggregates_descriptions, aggregate_overflow_row, settings.max_rows_to_group_by, @@ -1210,7 +1210,7 @@ void Planner::buildQueryPlanIfNeeded() settings.group_by_two_level_threshold_bytes, settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set - || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && aggregate_keys.empty() + || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && aggregation_keys.empty() && query_analyzer_const_aggregation_keys), planner_context->getQueryContext()->getTemporaryVolume(), settings.max_threads, diff --git a/src/Planner/PlannerAggregation.cpp b/src/Planner/PlannerAggregation.cpp new file mode 100644 index 00000000000..14b77faec8f --- /dev/null +++ b/src/Planner/PlannerAggregation.cpp @@ -0,0 +1,195 @@ +#include + +#include + +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +enum class GroupByKind +{ + ORDINARY, + ROLLUP, + CUBE, + GROUPING_SETS +}; + +class GroupingFunctionResolveMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + Data(GroupByKind group_by_kind_, + const Names & aggregation_keys_, + const GroupingSetsParamsList & grouping_sets_parameters_list_, + const PlannerContext & planner_context_) + : group_by_kind(group_by_kind_) + , planner_context(planner_context_) + { + size_t aggregation_keys_size = aggregation_keys_.size(); + for (size_t i = 0; i < aggregation_keys_size; ++i) + aggegation_key_to_index.emplace(aggregation_keys_[i], i); + + for (const auto & grouping_sets_parameter : grouping_sets_parameters_list_) + { + grouping_sets_keys_indices.emplace_back(); + auto & grouping_set_keys_indices = grouping_sets_keys_indices.back(); + + for (const auto & used_key : grouping_sets_parameter.used_keys) + { + auto aggregation_key_index_it = aggegation_key_to_index.find(used_key); + if (aggregation_key_index_it == aggegation_key_to_index.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Aggregation key {} in GROUPING SETS is not found in GROUP BY keys"); + + grouping_set_keys_indices.push_back(aggregation_key_index_it->second); + } + } + } + + GroupByKind group_by_kind; + std::unordered_map aggegation_key_to_index; + // Indexes of aggregation keys used in each grouping set (only for GROUP BY GROUPING SETS) + ColumnNumbersList grouping_sets_keys_indices; + const PlannerContext & planner_context; + }; + + static void visit(const QueryTreeNodePtr & node, Data & data) + { + auto * function_node = node->as(); + if (!function_node || function_node->getFunctionName() != "grouping") + return; + + size_t aggregation_keys_size = data.aggegation_key_to_index.size(); + + ColumnNumbers arguments_indexes; + + for (const auto & argument : function_node->getArguments().getNodes()) + { + String action_node_name = calculateActionNodeName(argument, data.planner_context); + + auto it = data.aggegation_key_to_index.find(action_node_name); + if (it == data.aggegation_key_to_index.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument of GROUPING function {} is not a part of GROUP BY clause", + argument->formatASTForErrorMessage()); + + arguments_indexes.push_back(it->second); + } + + QueryTreeNodeWeakPtr column_source; + auto grouping_set_argument_column = std::make_shared(NameAndTypePair{"__grouping_set", std::make_shared()}, column_source); + function_node->getArguments().getNodes().clear(); + + switch (data.group_by_kind) + { + case GroupByKind::ORDINARY: + { + auto grouping_ordinary_function = std::make_shared(arguments_indexes); + auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_ordinary_function)); + function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); + break; + } + case GroupByKind::ROLLUP: + { + auto grouping_ordinary_function = std::make_shared(arguments_indexes, aggregation_keys_size); + auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_ordinary_function)); + function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); + function_node->getArguments().getNodes().push_back(std::move(grouping_set_argument_column)); + break; + } + case GroupByKind::CUBE: + { + auto grouping_ordinary_function = std::make_shared(arguments_indexes, aggregation_keys_size); + auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_ordinary_function)); + function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); + function_node->getArguments().getNodes().push_back(std::move(grouping_set_argument_column)); + break; + } + case GroupByKind::GROUPING_SETS: + { + auto grouping_grouping_sets_function = std::make_shared(arguments_indexes, data.grouping_sets_keys_indices); + auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_grouping_sets_function)); + function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); + function_node->getArguments().getNodes().push_back(std::move(grouping_set_argument_column)); + break; + } + } + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) + { + return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; + } +}; + +using GroupingFunctionResolveVisitor = GroupingFunctionResolveMatcher::Visitor; + +void resolveGroupingFunctions(QueryTreeNodePtr & node, + GroupByKind group_by_kind, + const Names & aggregation_keys, + const GroupingSetsParamsList & grouping_sets_parameters_list, + const PlannerContext & planner_context) +{ + GroupingFunctionResolveVisitor::Data data {group_by_kind, aggregation_keys, grouping_sets_parameters_list, planner_context}; + GroupingFunctionResolveVisitor visitor(data); + visitor.visit(node); +} + +} + +void resolveGroupingFunctions(QueryTreeNodePtr & query_node, + const Names & aggregation_keys, + const GroupingSetsParamsList & grouping_sets_parameters_list, + const PlannerContext & planner_context) +{ + auto & query_node_typed = query_node->as(); + + GroupByKind group_by_kind = GroupByKind::ORDINARY; + if (query_node_typed.isGroupByWithRollup()) + group_by_kind = GroupByKind::ROLLUP; + else if (query_node_typed.isGroupByWithCube()) + group_by_kind = GroupByKind::CUBE; + else if (query_node_typed.isGroupByWithGroupingSets()) + group_by_kind = GroupByKind::GROUPING_SETS; + + if (query_node_typed.hasHaving()) + { + resolveGroupingFunctions(query_node_typed.getHaving(), + group_by_kind, + aggregation_keys, + grouping_sets_parameters_list, + planner_context); + } + + resolveGroupingFunctions(query_node_typed.getOrderByNode(), + group_by_kind, + aggregation_keys, + grouping_sets_parameters_list, + planner_context); + + resolveGroupingFunctions(query_node_typed.getProjectionNode(), + group_by_kind, + aggregation_keys, + grouping_sets_parameters_list, + planner_context); +} + +} diff --git a/src/Planner/PlannerAggregation.h b/src/Planner/PlannerAggregation.h new file mode 100644 index 00000000000..45858f03450 --- /dev/null +++ b/src/Planner/PlannerAggregation.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +#include + +#include + +namespace DB +{ + +/** Resolve GROUPING functions in query node. + * GROUPING function is replaced with specialized GROUPING function based on GROUP BY modifiers. + * For ROLLUP, CUBE, GROUPING SETS specialized GROUPING function take special __grouping_set column as argument. + */ +void resolveGroupingFunctions(QueryTreeNodePtr & query_node, + const Names & aggregation_keys, + const GroupingSetsParamsList & grouping_sets_parameters_list, + const PlannerContext & planner_context); + +} From 81e4c67f175a9cb278ab46c94cff174a16c490e9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 3 Sep 2022 18:20:09 +0200 Subject: [PATCH 089/188] Added INTERPOLATE support --- src/Analyzer/IQueryTreeNode.cpp | 1 + src/Analyzer/IQueryTreeNode.h | 1 + src/Analyzer/InterpolateColumnNode.cpp | 66 +++++++++++++++++++++++ src/Analyzer/InterpolateColumnNode.h | 69 ++++++++++++++++++++++++ src/Analyzer/QueryAnalysisPass.cpp | 31 ++++++++++- src/Analyzer/QueryNode.cpp | 15 ++++++ src/Analyzer/QueryNode.h | 20 ++++++- src/Analyzer/QueryTreeBuilder.cpp | 28 ++++++++++ src/Planner/Planner.cpp | 74 ++++++++++++++++++++++++-- src/Planner/PlannerActionsVisitor.cpp | 24 ++------- 10 files changed, 302 insertions(+), 27 deletions(-) create mode 100644 src/Analyzer/InterpolateColumnNode.cpp create mode 100644 src/Analyzer/InterpolateColumnNode.h diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 6643a254bab..7a8023b05e2 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -31,6 +31,7 @@ const char * toString(QueryTreeNodeType type) case QueryTreeNodeType::COLUMN: return "COLUMN"; case QueryTreeNodeType::LAMBDA: return "LAMBDA"; case QueryTreeNodeType::SORT_COLUMN: return "SORT_COLUMN"; + case QueryTreeNodeType::INTERPOLATE_COLUMN: return "INTERPOLATE_COLUMN"; case QueryTreeNodeType::TABLE: return "TABLE"; case QueryTreeNodeType::TABLE_FUNCTION: return "TABLE_FUNCTION"; case QueryTreeNodeType::QUERY: return "QUERY"; diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 3356fdb7b31..b720a4ff90c 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -37,6 +37,7 @@ enum class QueryTreeNodeType COLUMN, LAMBDA, SORT_COLUMN, + INTERPOLATE_COLUMN, TABLE, TABLE_FUNCTION, QUERY, diff --git a/src/Analyzer/InterpolateColumnNode.cpp b/src/Analyzer/InterpolateColumnNode.cpp new file mode 100644 index 00000000000..33e1821e9c1 --- /dev/null +++ b/src/Analyzer/InterpolateColumnNode.cpp @@ -0,0 +1,66 @@ +#include + +#include + +#include +#include + +#include + +namespace DB +{ + +InterpolateColumnNode::InterpolateColumnNode(QueryTreeNodePtr expression_, QueryTreeNodePtr interpolate_expression_) +{ + children.resize(children_size); + children[expression_child_index] = std::move(expression_); + children[interpolate_expression_child_index] = std::move(interpolate_expression_); +} + +String InterpolateColumnNode::getName() const +{ + String result = getExpression()->getName(); + result += " AS "; + result += getInterpolateExpression()->getName(); + + return result; +} + +void InterpolateColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const +{ + buffer << std::string(indent, ' ') << "INTERPOLATE_COLUMN id: " << format_state.getNodeId(this); + + buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION\n"; + getExpression()->dumpTreeImpl(buffer, format_state, indent + 4); + + buffer << '\n' << std::string(indent + 2, ' ') << "INTERPOLATE_EXPRESSION\n"; + getInterpolateExpression()->dumpTreeImpl(buffer, format_state, indent + 4); +} + +bool InterpolateColumnNode::isEqualImpl(const IQueryTreeNode &) const +{ + /// No state in interpolate column node + return true; +} + +void InterpolateColumnNode::updateTreeHashImpl(HashState &) const +{ + /// No state in interpolate column node +} + +ASTPtr InterpolateColumnNode::toASTImpl() const +{ + auto result = std::make_shared(); + result->column = getExpression()->toAST()->getColumnName(); + result->children.push_back(getInterpolateExpression()->toAST()); + result->expr = result->children.back(); + + return result; +} + +QueryTreeNodePtr InterpolateColumnNode::cloneImpl() const +{ + return std::make_shared(nullptr /*expression*/, nullptr /*interpolate_expression*/); +} + +} diff --git a/src/Analyzer/InterpolateColumnNode.h b/src/Analyzer/InterpolateColumnNode.h new file mode 100644 index 00000000000..6e2f35349ba --- /dev/null +++ b/src/Analyzer/InterpolateColumnNode.h @@ -0,0 +1,69 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/** Interpolate column node represents single column interpolation in INTERPOLATE section that part of ORDER BY in query tree. + * Example: SELECT * FROM test_table ORDER BY sort_column_1, sort_column_2; + */ +class InterpolateColumnNode; +using InterpolateColumnNodePtr = std::shared_ptr; + +class InterpolateColumnNode final : public IQueryTreeNode +{ +public: + /// Initialize interpolate column node with expression and interpolate expression + explicit InterpolateColumnNode(QueryTreeNodePtr expression_, QueryTreeNodePtr interpolate_expression_); + + /// Get expression + const QueryTreeNodePtr & getExpression() const + { + return children[expression_child_index]; + } + + /// Get expression + QueryTreeNodePtr & getExpression() + { + return children[expression_child_index]; + } + + /// Get expression + const QueryTreeNodePtr & getInterpolateExpression() const + { + return children[interpolate_expression_child_index]; + } + + /// Get expression + QueryTreeNodePtr & getInterpolateExpression() + { + return children[interpolate_expression_child_index]; + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::INTERPOLATE_COLUMN; + } + + String getName() const override; + + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + + void updateTreeHashImpl(HashState & hash_state) const override; + +protected: + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + static constexpr size_t expression_child_index = 0; + static constexpr size_t interpolate_expression_child_index = 1; + static constexpr size_t children_size = interpolate_expression_child_index + 1; +}; + +} diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index dc7a8994da7..ef8e4ebeced 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -41,6 +41,7 @@ #include #include #include +#include #include #include #include @@ -885,6 +886,8 @@ private: void resolveSortColumnsNodeList(QueryTreeNodePtr & sort_columns_node_list, IdentifierResolveScope & scope); + void resolveInterpolateColumnsNodeList(QueryTreeNodePtr & sort_columns_node_list, IdentifierResolveScope & scope); + String calculateProjectionNodeDisplayName(QueryTreeNodePtr & node, IdentifierResolveScope & scope); NamesAndTypes resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope); @@ -3168,7 +3171,6 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes /// Lambda must be resolved by caller break; } - case QueryTreeNodeType::SORT_COLUMN: { throw Exception(ErrorCodes::LOGICAL_ERROR, @@ -3176,6 +3178,13 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } + case QueryTreeNodeType::INTERPOLATE_COLUMN: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Interpolate column {} is not allowed in expression. In scope {}", + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } case QueryTreeNodeType::TABLE: { if (!allow_table_expression) @@ -3341,6 +3350,20 @@ void QueryAnalyzer::resolveSortColumnsNodeList(QueryTreeNodePtr & sort_columns_n } } +/** Resolve interpolate columns nodes list. + */ +void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & sort_columns_node_list, IdentifierResolveScope & scope) +{ + auto & sort_columns_node_list_typed = sort_columns_node_list->as(); + for (auto & node : sort_columns_node_list_typed.getNodes()) + { + auto & interpolate_column_node = node->as(); + + resolveExpressionNode(interpolate_column_node.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + resolveExpressionNode(interpolate_column_node.getInterpolateExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + } +} + class SubqueryToProjectionNameMatcher { public: @@ -4321,6 +4344,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasOrderBy()) visitor.visit(query_node_typed.getOrderByNode()); + if (query_node_typed.hasInterpolate()) + visitor.visit(query_node_typed.getInterpolate()); + /// Register CTE subqueries and remove them from WITH section auto & with_nodes = query_node_typed.getWith().getNodes(); @@ -4407,6 +4433,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasOrderBy()) resolveSortColumnsNodeList(query_node_typed.getOrderByNode(), scope); + if (query_node_typed.hasInterpolate()) + resolveInterpolateColumnsNodeList(query_node_typed.getInterpolate(), scope); + if (query_node_typed.hasLimit()) { resolveExpressionNode(query_node_typed.getLimit(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 6b23c34ca4f..e1dbeb37c1b 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -80,6 +80,12 @@ String QueryNode::getName() const buffer << getOrderByNode()->getName(); } + if (hasInterpolate()) + { + buffer << " INTERPOLATE"; + buffer << getInterpolate()->getName(); + } + if (hasLimit()) { buffer << " LIMIT "; @@ -181,6 +187,12 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s getOrderBy().dumpTreeImpl(buffer, format_state, indent + 4); } + if (hasInterpolate()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "INTERPOLATE\n"; + getInterpolate()->dumpTreeImpl(buffer, format_state, indent + 4); + } + if (hasLimit()) { buffer << '\n' << std::string(indent + 2, ' ') << "LIMIT\n"; @@ -276,6 +288,9 @@ ASTPtr QueryNode::toASTImpl() const if (hasOrderBy()) select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, getOrderBy().toAST()); + if (hasInterpolate()) + select_query->setExpression(ASTSelectQuery::Expression::INTERPOLATE, getInterpolate()->toAST()); + if (hasLimit()) select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, getLimit()->toAST()); diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 420356a4a0a..5f89ad93f47 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -260,6 +260,21 @@ public: return children[order_by_child_index]; } + bool hasInterpolate() const + { + return getInterpolate() != nullptr; + } + + const QueryTreeNodePtr & getInterpolate() const + { + return children[interpolate_child_index]; + } + + QueryTreeNodePtr & getInterpolate() + { + return children[interpolate_child_index]; + } + bool hasLimit() const { return children[limit_child_index] != nullptr; @@ -359,8 +374,9 @@ private: static constexpr size_t group_by_child_index = 5; static constexpr size_t having_child_index = 6; static constexpr size_t order_by_child_index = 7; - static constexpr size_t limit_child_index = 8; - static constexpr size_t offset_child_index = 9; + static constexpr size_t interpolate_child_index = 8; + static constexpr size_t limit_child_index = 9; + static constexpr size_t offset_child_index = 10; static constexpr size_t children_size = offset_child_index + 1; }; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 5f096440885..588332b3e18 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include @@ -31,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -82,6 +84,8 @@ private: QueryTreeNodePtr buildSortColumnList(const ASTPtr & order_by_expression_list) const; + QueryTreeNodePtr buildInterpolateColumnList(const ASTPtr & interpolate_expression_list) const; + QueryTreeNodePtr buildExpressionList(const ASTPtr & expression_list) const; QueryTreeNodePtr buildExpression(const ASTPtr & expression) const; @@ -248,6 +252,10 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q if (select_order_by_list) current_query_tree->getOrderByNode() = buildSortColumnList(select_order_by_list); + auto interpolate_list = select_query_typed.interpolate(); + if (interpolate_list) + current_query_tree->getInterpolate() = buildInterpolateColumnList(interpolate_list); + auto select_limit = select_query_typed.limitLength(); if (select_limit) current_query_tree->getLimit() = buildExpression(select_limit); @@ -296,6 +304,26 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortColumnList(const ASTPtr & order_by_e return list_node; } +QueryTreeNodePtr QueryTreeBuilder::buildInterpolateColumnList(const ASTPtr & interpolate_expression_list) const +{ + auto list_node = std::make_shared(); + + auto & expression_list_typed = interpolate_expression_list->as(); + list_node->getNodes().reserve(expression_list_typed.children.size()); + + for (auto & expression : expression_list_typed.children) + { + const auto & interpolate_element = expression->as(); + auto expression_to_interpolate = std::make_shared(Identifier(interpolate_element.column)); + auto interpolate_expression = buildExpression(interpolate_element.expr); + auto interpolate_column_node = std::make_shared(std::move(expression_to_interpolate), std::move(interpolate_expression)); + + list_node->getNodes().push_back(std::move(interpolate_column_node)); + } + + return list_node; +} + QueryTreeNodePtr QueryTreeBuilder::buildExpressionList(const ASTPtr & expression_list) const { auto list_node = std::make_shared(); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 1173ae48604..036be3fd50f 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -45,6 +45,7 @@ #include #include #include +#include #include #include #include @@ -98,9 +99,10 @@ namespace ErrorCodes * TODO: Interpreter resources * TODO: Support max streams * TODO: Support GROUPINS SETS, const aggregation keys - * TODO: Support interpolate, LIMIT BY. + * TODO: Support LIMIT BY * TODO: Support ORDER BY read in order optimization * TODO: Support GROUP BY read in order optimization + * TODO: Support GROUP BY constant keys */ namespace @@ -1099,15 +1101,24 @@ void Planner::buildQueryPlanIfNeeded() auto & actions_dag_outputs = actions_dag->getOutputs(); actions_dag_outputs.clear(); + std::unordered_set order_by_actions_dag_outputs_node_names; + /** We add only sort column sort expression in before ORDER BY actions DAG. * WITH fill expressions must be constant nodes. */ - auto & order_by_node_list = query_node.getOrderByNode()->as(); + auto & order_by_node_list = query_node.getOrderBy(); for (auto & sort_column_node : order_by_node_list.getNodes()) { auto & sort_column_node_typed = sort_column_node->as(); - auto expression_dag_index_nodes = actions_visitor.visit(actions_dag, sort_column_node_typed.getExpression()); - actions_dag_outputs.insert(actions_dag_outputs.end(), expression_dag_index_nodes.begin(), expression_dag_index_nodes.end()); + auto expression_dag_nodes = actions_visitor.visit(actions_dag, sort_column_node_typed.getExpression()); + for (auto & action_dag_node : expression_dag_nodes) + { + if (order_by_actions_dag_outputs_node_names.contains(action_dag_node->result_name)) + continue; + + actions_dag_outputs.push_back(action_dag_node); + order_by_actions_dag_outputs_node_names.insert(action_dag_node->result_name); + } } auto actions_step_before_order_by = std::make_unique(std::move(actions_dag)); @@ -1376,16 +1387,69 @@ void Planner::buildQueryPlanIfNeeded() sorting_step->setStepDescription("Sorting for ORDER BY"); query_plan.addStep(std::move(sorting_step)); + NameSet column_names_with_fill; SortDescription fill_description; for (auto & description : sort_description) { if (description.with_fill) + { fill_description.push_back(description); + column_names_with_fill.insert(description.column_name); + } } if (!fill_description.empty()) { - InterpolateDescriptionPtr interpolate_description = nullptr; + InterpolateDescriptionPtr interpolate_description; + + if (query_node.hasInterpolate()) + { + auto interpolate_actions_dag = std::make_shared(); + + auto & interpolate_column_list_node = query_node.getInterpolate()->as(); + auto & interpolate_column_list_nodes = interpolate_column_list_node.getNodes(); + if (interpolate_column_list_nodes.empty()) + { + auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + for (auto & query_plan_column : query_plan_columns) + { + if (column_names_with_fill.contains(query_plan_column.name)) + continue; + + const auto * input_action_node = &interpolate_actions_dag->addInput(query_plan_column); + interpolate_actions_dag->getOutputs().push_back(input_action_node); + } + } + else + { + for (auto & interpolate_column_node : interpolate_column_list_node.getNodes()) + { + auto & interpolate_column_node_typed = interpolate_column_node->as(); + auto expression_to_interpolate_expression_nodes = actions_visitor.visit(interpolate_actions_dag, interpolate_column_node_typed.getExpression()); + auto interpolate_expression_nodes = actions_visitor.visit(interpolate_actions_dag, interpolate_column_node_typed.getInterpolateExpression()); + + if (expression_to_interpolate_expression_nodes.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression to interpolate expected to have single action node"); + + if (interpolate_expression_nodes.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interpolate expression expected to have single action node"); + + const auto * expression_to_interpolate = expression_to_interpolate_expression_nodes[0]; + const auto & alias_name = expression_to_interpolate->result_name; + + const auto * interpolate_expression = interpolate_expression_nodes[0]; + const auto * alias_node = &interpolate_actions_dag->addAlias(*interpolate_expression, alias_name); + + interpolate_actions_dag->getOutputs().push_back(alias_node); + } + + interpolate_actions_dag->removeUnusedActions(); + } + + Aliases empty_aliases; + interpolate_description = std::make_shared(std::move(interpolate_actions_dag), empty_aliases); + } + auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_description), interpolate_description); query_plan.addStep(std::move(filling_step)); } diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index ded4b30204e..774f0c4babe 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -239,8 +239,10 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi { actions_stack[i].addInputColumnIfNecessary(column_node_name, column_node.getColumnType()); - if (column_node.getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA - && actions_stack[i].getScopeNode().get() == column_node.getColumnSource().get()) + auto column_source = column_node.getColumnSourceOrNull(); + if (column_source && + column_source->getNodeType() == QueryTreeNodeType::LAMBDA && + actions_stack[i].getScopeNode().get() == column_source.get()) { return {column_node_name, i}; } @@ -384,24 +386,8 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi std::optional in_function_second_argument_node_name_with_level; - if (function_node.getFunctionName() == "grouping") - { - size_t arguments_size = function_node.getArguments().getNodes().size(); - - if (arguments_size == 0) - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, - "Function GROUPING expects at least one argument"); - else if (arguments_size > 64) - throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, - "Function GROUPING can have up to 64 arguments, but {} provided", - arguments_size); - - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function GROUPING is not supported"); - } - else if (isNameOfInFunction(function_node.getFunctionName())) - { + if (isNameOfInFunction(function_node.getFunctionName())) in_function_second_argument_node_name_with_level = makeSetForInFunction(node); - } const auto & function_arguments = function_node.getArguments().getNodes(); size_t function_arguments_size = function_arguments.size(); From 4110460bcd204fd4433c056cbdb31f1f047f1bd7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 3 Sep 2022 19:21:17 +0200 Subject: [PATCH 090/188] Added LIMIT BY support --- src/Analyzer/QueryAnalysisPass.cpp | 77 ++++++++++++++++++++---------- src/Analyzer/QueryNode.cpp | 48 ++++++++++++++++++- src/Analyzer/QueryNode.h | 62 +++++++++++++++++++++++- src/Analyzer/QueryTreeBuilder.cpp | 12 +++++ src/Planner/Planner.cpp | 59 ++++++++++++++++++++--- 5 files changed, 224 insertions(+), 34 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index ef8e4ebeced..2cf0cd72ffa 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -838,6 +838,8 @@ private: void evaluateScalarSubquery(QueryTreeNodePtr & query_tree_node, size_t subquery_depth); + static void validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope); + /// Resolve identifier functions QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier); @@ -1067,6 +1069,23 @@ void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node, size_t subqu union_node->performConstantFolding(std::move(constant_value)); } +void QueryAnalyzer::validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope) +{ + const auto limit_offset_constant_value = expression_node->getConstantValueOrNull(); + if (!limit_offset_constant_value || !isNativeNumber(removeNullable(limit_offset_constant_value->getType()))) + throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, + "{} expression must be constant with numeric type. Actual {}. In scope {}", + expression_description, + expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + Field converted = convertFieldToType(limit_offset_constant_value->getValue(), DataTypeUInt64()); + if (converted.isNull()) + throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, + "{} numeric constant expression is not representable as UInt64", + expression_description); +} + /// Resolve identifier functions implementation /// Try resolve table identifier from database catalog @@ -4347,6 +4366,21 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasInterpolate()) visitor.visit(query_node_typed.getInterpolate()); + if (query_node_typed.hasLimitByLimit()) + visitor.visit(query_node_typed.getLimitByLimit()); + + if (query_node_typed.hasLimitByOffset()) + visitor.visit(query_node_typed.getLimitByOffset()); + + if (query_node_typed.hasLimitBy()) + visitor.visit(query_node_typed.getLimitByNode()); + + if (query_node_typed.hasLimit()) + visitor.visit(query_node_typed.getLimit()); + + if (query_node_typed.hasOffset()) + visitor.visit(query_node_typed.getOffset()); + /// Register CTE subqueries and remove them from WITH section auto & with_nodes = query_node_typed.getWith().getNodes(); @@ -4436,38 +4470,31 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasInterpolate()) resolveInterpolateColumnsNodeList(query_node_typed.getInterpolate(), scope); + if (query_node_typed.hasLimitByLimit()) + { + resolveExpressionNode(query_node_typed.getLimitByLimit(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + validateLimitOffsetExpression(query_node_typed.getLimitByLimit(), "LIMIT BY LIMIT", scope); + } + + if (query_node_typed.hasLimitByOffset()) + { + resolveExpressionNode(query_node_typed.getLimitByOffset(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + validateLimitOffsetExpression(query_node_typed.getLimitByOffset(), "LIMIT BY OFFSET", scope); + } + + if (query_node_typed.hasLimitBy()) + resolveExpressionNodeList(query_node_typed.getLimitByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + if (query_node_typed.hasLimit()) { resolveExpressionNode(query_node_typed.getLimit(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - - const auto limit_constant_value = query_node_typed.getLimit()->getConstantValueOrNull(); - if (!limit_constant_value || !isNativeNumber(removeNullable(limit_constant_value->getType()))) - throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, - "Limit expression must be constant with numeric type. Actual {}. In scope {}", - query_node_typed.getLimit()->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - - Field converted = convertFieldToType(limit_constant_value->getValue(), DataTypeUInt64()); - if (converted.isNull()) - throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, - "Limit numeric constant expression is not representable as UInt64"); + validateLimitOffsetExpression(query_node_typed.getLimit(), "LIMIT", scope); } - if (query_node_typed.getOffset()) + if (query_node_typed.hasOffset()) { resolveExpressionNode(query_node_typed.getOffset(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - - const auto offset_constant_value = query_node_typed.getOffset()->getConstantValueOrNull(); - if (!offset_constant_value || !isNativeNumber(removeNullable(offset_constant_value->getType()))) - throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, - "Offset expression must be constant with numeric type. Actual {}. In scope {}", - query_node_typed.getLimit()->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - - Field converted = convertFieldToType(offset_constant_value->getValue(), DataTypeUInt64()); - if (converted.isNull()) - throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, - "Offset numeric constant expression is not representable as UInt64"); + validateLimitOffsetExpression(query_node_typed.getOffset(), "OFFSET", scope); } /** WITH section can be safely removed, because WITH section only can provide aliases to query expressions diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index e1dbeb37c1b..e857b35acd9 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -29,6 +29,7 @@ QueryNode::QueryNode() children[projection_child_index] = std::make_shared(); children[group_by_child_index] = std::make_shared(); children[order_by_child_index] = std::make_shared(); + children[limit_by_child_index] = std::make_shared(); } String QueryNode::getName() const @@ -82,10 +83,28 @@ String QueryNode::getName() const if (hasInterpolate()) { - buffer << " INTERPOLATE"; + buffer << " INTERPOLATE "; buffer << getInterpolate()->getName(); } + if (hasLimitByLimit()) + { + buffer << "LIMIT "; + buffer << getLimitByLimit()->getName(); + } + + if (hasLimitByOffset()) + { + buffer << "OFFSET "; + buffer << getLimitByOffset()->getName(); + } + + if (hasLimitBy()) + { + buffer << " BY "; + buffer << getLimitBy().getName(); + } + if (hasLimit()) { buffer << " LIMIT "; @@ -193,6 +212,24 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s getInterpolate()->dumpTreeImpl(buffer, format_state, indent + 4); } + if (hasLimitByLimit()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "LIMIT BY LIMIT\n"; + getLimitByLimit()->dumpTreeImpl(buffer, format_state, indent + 4); + } + + if (hasLimitByOffset()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "LIMIT BY OFFSET\n"; + getLimitByOffset()->dumpTreeImpl(buffer, format_state, indent + 4); + } + + if (hasLimitBy()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "LIMIT BY\n"; + getLimitBy().dumpTreeImpl(buffer, format_state, indent + 4); + } + if (hasLimit()) { buffer << '\n' << std::string(indent + 2, ' ') << "LIMIT\n"; @@ -291,6 +328,15 @@ ASTPtr QueryNode::toASTImpl() const if (hasInterpolate()) select_query->setExpression(ASTSelectQuery::Expression::INTERPOLATE, getInterpolate()->toAST()); + if (hasLimitByLimit()) + select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY_LENGTH, getLimitByLimit()->toAST()); + + if (hasLimitByOffset()) + select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY_OFFSET, getLimitByOffset()->toAST()); + + if (hasLimitBy()) + select_query->setExpression(ASTSelectQuery::Expression::LIMIT_BY, getLimitBy().toAST()); + if (hasLimit()) select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, getLimit()->toAST()); diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 5f89ad93f47..4dc893fcfa7 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -275,6 +275,61 @@ public: return children[interpolate_child_index]; } + bool hasLimitByLimit() const + { + return children[limit_by_limit_child_index] != nullptr; + } + + const QueryTreeNodePtr & getLimitByLimit() const + { + return children[limit_by_limit_child_index]; + } + + QueryTreeNodePtr & getLimitByLimit() + { + return children[limit_by_limit_child_index]; + } + + bool hasLimitByOffset() const + { + return children[limit_by_offset_child_index] != nullptr; + } + + const QueryTreeNodePtr & getLimitByOffset() const + { + return children[limit_by_offset_child_index]; + } + + QueryTreeNodePtr & getLimitByOffset() + { + return children[limit_by_offset_child_index]; + } + + bool hasLimitBy() const + { + return !getLimitBy().getNodes().empty(); + } + + const ListNode & getLimitBy() const + { + return children[limit_by_child_index]->as(); + } + + ListNode & getLimitBy() + { + return children[limit_by_child_index]->as(); + } + + const QueryTreeNodePtr & getLimitByNode() const + { + return children[limit_by_child_index]; + } + + QueryTreeNodePtr & getLimitByNode() + { + return children[limit_by_child_index]; + } + bool hasLimit() const { return children[limit_child_index] != nullptr; @@ -375,8 +430,11 @@ private: static constexpr size_t having_child_index = 6; static constexpr size_t order_by_child_index = 7; static constexpr size_t interpolate_child_index = 8; - static constexpr size_t limit_child_index = 9; - static constexpr size_t offset_child_index = 10; + static constexpr size_t limit_by_limit_child_index = 9; + static constexpr size_t limit_by_offset_child_index = 10; + static constexpr size_t limit_by_child_index = 11; + static constexpr size_t limit_child_index = 12; + static constexpr size_t offset_child_index = 13; static constexpr size_t children_size = offset_child_index + 1; }; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 588332b3e18..8533e6a12c8 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -256,6 +256,18 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q if (interpolate_list) current_query_tree->getInterpolate() = buildInterpolateColumnList(interpolate_list); + auto select_limit_by_limit = select_query_typed.limitByLength(); + if (select_limit_by_limit) + current_query_tree->getLimitByLimit() = buildExpression(select_limit_by_limit); + + auto select_limit_by_offset = select_query_typed.limitOffset(); + if (select_limit_by_offset) + current_query_tree->getLimitByOffset() = buildExpression(select_limit_by_offset); + + auto select_limit_by = select_query_typed.limitBy(); + if (select_limit_by) + current_query_tree->getLimitByNode() = buildExpressionList(select_limit_by); + auto select_limit = select_query_typed.limitLength(); if (select_limit) current_query_tree->getLimit() = buildExpression(select_limit); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 036be3fd50f..7f7783c3b1f 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include @@ -1126,6 +1127,24 @@ void Planner::buildQueryPlanIfNeeded() before_order_by_step_index = actions_chain.getLastStepIndex(); } + std::optional before_limit_by_step_index; + Names limit_by_columns_names; + + if (query_node.hasLimitBy()) + { + chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & limit_by_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + auto limit_by_actions = convertExpressionNodeIntoDAG(query_node.getLimitByNode(), limit_by_input, planner_context); + + limit_by_columns_names.reserve(limit_by_actions->getOutputs().size()); + for (auto & output_node : limit_by_actions->getOutputs()) + limit_by_columns_names.push_back(output_node->result_name); + + auto actions_step_before_limit_by = std::make_unique(std::move(limit_by_actions)); + actions_chain.addStep(std::move(actions_step_before_limit_by)); + before_limit_by_step_index = actions_chain.getLastStepIndex(); + } + chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); auto projection_actions = convertExpressionNodeIntoDAG(query_node.getProjectionNode(), projection_input, planner_context); @@ -1353,13 +1372,14 @@ void Planner::buildQueryPlanIfNeeded() if (before_order_by_step_index) { - auto & aggregate_actions_chain_node = actions_chain.at(*before_order_by_step_index); + auto & before_order_by_actions_chain_node = actions_chain.at(*before_order_by_step_index); auto expression_step_before_order_by = std::make_unique(query_plan.getCurrentDataStream(), - aggregate_actions_chain_node->getActions()); + before_order_by_actions_chain_node->getActions()); expression_step_before_order_by->setStepDescription("Before ORDER BY"); query_plan.addStep(std::move(expression_step_before_order_by)); } + QueryPlanStepPtr filling_step; SortDescription sort_description; if (query_node.hasOrderBy()) @@ -1450,11 +1470,38 @@ void Planner::buildQueryPlanIfNeeded() interpolate_description = std::make_shared(std::move(interpolate_actions_dag), empty_aliases); } - auto filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_description), interpolate_description); - query_plan.addStep(std::move(filling_step)); + filling_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(fill_description), interpolate_description); } } + if (before_limit_by_step_index) + { + auto & before_limit_by_actions_chain_node = actions_chain.at(*before_limit_by_step_index); + auto expression_step_before_limit_by = std::make_unique(query_plan.getCurrentDataStream(), + before_limit_by_actions_chain_node->getActions()); + expression_step_before_limit_by->setStepDescription("Before LIMIT BY"); + query_plan.addStep(std::move(expression_step_before_limit_by)); + } + + if (query_node.hasLimitByLimit() && query_node.hasLimitBy()) + { + /// Constness of LIMIT BY limit is validated during query analysis stage + UInt64 limit_by_limit = query_node.getLimitByLimit()->getConstantValue().getValue().safeGet(); + UInt64 limit_by_offset = 0; + + if (query_node.hasLimitByOffset()) + { + /// Constness of LIMIT BY offset is validated during query analysis stage + limit_by_offset = query_node.getLimitByOffset()->getConstantValue().getValue().safeGet(); + } + + auto limit_by_step = std::make_unique(query_plan.getCurrentDataStream(), limit_by_limit, limit_by_offset, limit_by_columns_names); + query_plan.addStep(std::move(limit_by_step)); + } + + if (filling_step) + query_plan.addStep(std::move(filling_step)); + if (query_context->getSettingsRef().extremes) { auto extremes_step = std::make_unique(query_plan.getCurrentDataStream()); @@ -1464,7 +1511,7 @@ void Planner::buildQueryPlanIfNeeded() UInt64 limit_offset = 0; if (query_node.hasOffset()) { - /// Validated during query analysis stage + /// Constness of offset is validated during query analysis stage limit_offset = query_node.getOffset()->getConstantValue().getValue().safeGet(); } @@ -1474,7 +1521,7 @@ void Planner::buildQueryPlanIfNeeded() bool always_read_till_end = settings.exact_rows_before_limit; bool limit_with_ties = query_node.isLimitWithTies(); - /// Validated during query analysis stage + /// Constness of limit is validated during query analysis stage UInt64 limit_length = query_node.getLimit()->getConstantValue().getValue().safeGet(); SortDescription limit_with_ties_sort_description; From 64c60718f8e7b04011c79eeff4c299d1f4fb6455 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 3 Sep 2022 19:34:52 +0200 Subject: [PATCH 091/188] Updated SORT COLUMN with fill --- src/Analyzer/InterpolateColumnNode.h | 4 +++- src/Analyzer/QueryTreeBuilder.cpp | 6 +++++- src/Analyzer/SortColumnNode.cpp | 16 +++++++++++----- src/Analyzer/SortColumnNode.h | 8 +++++--- src/Planner/Planner.cpp | 4 +--- src/Planner/PlannerSorting.cpp | 2 +- 6 files changed, 26 insertions(+), 14 deletions(-) diff --git a/src/Analyzer/InterpolateColumnNode.h b/src/Analyzer/InterpolateColumnNode.h index 6e2f35349ba..1ff66710898 100644 --- a/src/Analyzer/InterpolateColumnNode.h +++ b/src/Analyzer/InterpolateColumnNode.h @@ -7,7 +7,9 @@ namespace DB { /** Interpolate column node represents single column interpolation in INTERPOLATE section that part of ORDER BY in query tree. - * Example: SELECT * FROM test_table ORDER BY sort_column_1, sort_column_2; + * Example: SELECT * FROM test_table ORDER BY id WITH FILL INTERPOLATE value AS value + 1; + * value - expression to interpolate. + * value + 1 - interpolate expression. */ class InterpolateColumnNode; using InterpolateColumnNodePtr = std::shared_ptr; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 8533e6a12c8..27861fd38d8 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -301,7 +301,11 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortColumnList(const ASTPtr & order_by_e const auto & sort_expression_ast = order_by_element.children.at(0); auto sort_expression = buildExpression(sort_expression_ast); - auto sort_column_node = std::make_shared(std::move(sort_expression), sort_direction, nulls_sort_direction, std::move(collator)); + auto sort_column_node = std::make_shared(std::move(sort_expression), + sort_direction, + nulls_sort_direction, + std::move(collator), + order_by_element.with_fill); if (order_by_element.fill_from) sort_column_node->getFillFrom() = buildExpression(order_by_element.fill_from); diff --git a/src/Analyzer/SortColumnNode.cpp b/src/Analyzer/SortColumnNode.cpp index c55946db4f8..c3b9a98eb07 100644 --- a/src/Analyzer/SortColumnNode.cpp +++ b/src/Analyzer/SortColumnNode.cpp @@ -24,10 +24,12 @@ const char * toString(SortDirection sort_direction) SortColumnNode::SortColumnNode(QueryTreeNodePtr expression_, SortDirection sort_direction_, std::optional nulls_sort_direction_, - std::shared_ptr collator_) + std::shared_ptr collator_, + bool with_fill_) : sort_direction(sort_direction_) , nulls_sort_direction(nulls_sort_direction_) , collator(std::move(collator_)) + , with_fill(with_fill_) { children.resize(children_size); children[sort_expression_child_index] = std::move(expression_); @@ -50,7 +52,7 @@ String SortColumnNode::getName() const result += " NULLS LAST"; } - if (hasWithFill()) + if (with_fill) result += " WITH FILL"; if (hasFillFrom()) @@ -76,6 +78,8 @@ void SortColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_sta if (collator) buffer << ", collator: " << collator->getLocale(); + buffer << ", with_fill: " << with_fill; + buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION\n"; getExpression()->dumpTreeImpl(buffer, format_state, indent + 4); @@ -102,7 +106,8 @@ bool SortColumnNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); if (sort_direction != rhs_typed.sort_direction || - nulls_sort_direction != rhs_typed.nulls_sort_direction) + nulls_sort_direction != rhs_typed.nulls_sort_direction || + with_fill != rhs_typed.with_fill) return false; if (!collator && !rhs_typed.collator) @@ -119,6 +124,7 @@ void SortColumnNode::updateTreeHashImpl(HashState & hash_state) const { hash_state.update(sort_direction); hash_state.update(nulls_sort_direction); + hash_state.update(with_fill); if (collator) { @@ -141,7 +147,7 @@ ASTPtr SortColumnNode::toASTImpl() const if (collator) result->collation = std::make_shared(Field(collator->getLocale())); - result->with_fill = hasWithFill(); + result->with_fill = with_fill; result->fill_from = hasFillFrom() ? getFillFrom()->toAST() : nullptr; result->fill_to = hasFillTo() ? getFillTo()->toAST() : nullptr; result->fill_step = hasFillStep() ? getFillStep()->toAST() : nullptr; @@ -152,7 +158,7 @@ ASTPtr SortColumnNode::toASTImpl() const QueryTreeNodePtr SortColumnNode::cloneImpl() const { - return std::make_shared(nullptr, sort_direction, nulls_sort_direction, collator); + return std::make_shared(nullptr /*expression*/, sort_direction, nulls_sort_direction, collator, with_fill); } } diff --git a/src/Analyzer/SortColumnNode.h b/src/Analyzer/SortColumnNode.h index 04642a346a2..9ec73cadc4b 100644 --- a/src/Analyzer/SortColumnNode.h +++ b/src/Analyzer/SortColumnNode.h @@ -30,7 +30,8 @@ public: explicit SortColumnNode(QueryTreeNodePtr expression_, SortDirection sort_direction_ = SortDirection::ASCENDING, std::optional nulls_sort_direction_ = {}, - std::shared_ptr collator_ = nullptr); + std::shared_ptr collator_ = nullptr, + bool with_fill = false); /// Get sort expression const QueryTreeNodePtr & getExpression() const @@ -45,9 +46,9 @@ public: } /// Has with fill - bool hasWithFill() const + bool withFill() const { - return hasFillFrom() || hasFillStep() || hasFillTo(); + return with_fill; } /// Has fill from @@ -147,6 +148,7 @@ private: SortDirection sort_direction = SortDirection::ASCENDING; std::optional nulls_sort_direction; std::shared_ptr collator; + bool with_fill = false; }; } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 7f7783c3b1f..e4c414b410e 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -99,11 +99,9 @@ namespace ErrorCodes * TODO: UNION storage limits * TODO: Interpreter resources * TODO: Support max streams - * TODO: Support GROUPINS SETS, const aggregation keys - * TODO: Support LIMIT BY + * TODO: Support GROUP BY constant keys * TODO: Support ORDER BY read in order optimization * TODO: Support GROUP BY read in order optimization - * TODO: Support GROUP BY constant keys */ namespace diff --git a/src/Planner/PlannerSorting.cpp b/src/Planner/PlannerSorting.cpp index 301734e11ec..17304fa7766 100644 --- a/src/Planner/PlannerSorting.cpp +++ b/src/Planner/PlannerSorting.cpp @@ -135,7 +135,7 @@ SortDescription extractSortDescription(const QueryTreeNodePtr & order_by_node, c if (nulls_sort_direction) nulls_direction = *nulls_sort_direction == SortDirection::ASCENDING ? 1 : -1; - if (sort_column_node_typed.hasWithFill()) + if (sort_column_node_typed.withFill()) { FillColumnDescription fill_description = extractWithFillDescription(sort_column_node_typed); sort_column_description.emplace_back(column_name, direction, nulls_direction, collator, true /*with_fill*/, fill_description); From 51282621290aebb1c1af77491569e8f431bef37d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 3 Sep 2022 19:54:15 +0200 Subject: [PATCH 092/188] Added GROUP BY with constant keys support --- src/Planner/Planner.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index e4c414b410e..565dcb1a6b3 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -99,7 +99,6 @@ namespace ErrorCodes * TODO: UNION storage limits * TODO: Interpreter resources * TODO: Support max streams - * TODO: Support GROUP BY constant keys * TODO: Support ORDER BY read in order optimization * TODO: Support GROUP BY read in order optimization */ @@ -973,6 +972,7 @@ void Planner::buildQueryPlanIfNeeded() PlannerActionsVisitor actions_visitor(planner_context); GroupingSetsParamsList grouping_sets_parameters_list; + bool group_by_with_constant_keys = false; if (query_node.hasGroupBy()) { @@ -986,6 +986,8 @@ void Planner::buildQueryPlanIfNeeded() for (auto & grouping_set_key_node : grouping_set_keys_list_node_typed.getNodes()) { + group_by_with_constant_keys |= grouping_set_key_node->hasConstantValue(); + auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, grouping_set_key_node); aggregation_keys.reserve(expression_dag_nodes.size()); @@ -1027,6 +1029,9 @@ void Planner::buildQueryPlanIfNeeded() } else { + for (auto & group_by_key_node : query_node.getGroupBy().getNodes()) + group_by_with_constant_keys |= group_by_key_node->hasConstantValue(); + auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, query_node.getGroupByNode()); aggregation_keys.reserve(expression_dag_nodes.size()); @@ -1214,7 +1219,6 @@ void Planner::buildQueryPlanIfNeeded() } const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); - bool query_analyzer_const_aggregation_keys = false; const auto stats_collecting_params = Aggregator::Params::StatsCollectingParams( select_query_info.query, @@ -1239,7 +1243,7 @@ void Planner::buildQueryPlanIfNeeded() settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && aggregation_keys.empty() - && query_analyzer_const_aggregation_keys), + && group_by_with_constant_keys), planner_context->getQueryContext()->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data, From fed146e1980d78d498906f0948a230892f958e5f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 4 Sep 2022 17:20:59 +0200 Subject: [PATCH 093/188] Added SAMPLE BY support. Added SAMPLE BY, FINAL support for JOINS. --- src/Analyzer/IdentifierNode.cpp | 18 +++ src/Analyzer/IdentifierNode.h | 24 ++++ src/Analyzer/QueryAnalysisPass.cpp | 120 ++++++++++++++++-- src/Analyzer/QueryAnalysisPass.h | 1 + src/Analyzer/QueryNode.cpp | 18 +++ src/Analyzer/QueryNode.h | 20 +++ src/Analyzer/QueryTreeBuilder.cpp | 54 +++++++- src/Analyzer/TableExpressionModifiers.cpp | 42 ++++++ src/Analyzer/TableExpressionModifiers.h | 71 +++++++++++ src/Analyzer/TableFunctionNode.cpp | 17 +++ src/Analyzer/TableFunctionNode.h | 22 +++- src/Analyzer/TableNode.cpp | 18 +++ src/Analyzer/TableNode.h | 20 +++ src/Analyzer/UnionNode.cpp | 17 +++ src/Analyzer/UnionNode.h | 21 +++ src/Analyzer/Utils.cpp | 43 ++++++- src/Parsers/ASTSampleRatio.h | 10 ++ src/Planner/Planner.cpp | 11 +- .../QueryPlan/ReadFromMergeTree.cpp | 20 ++- src/Storages/MergeTree/KeyCondition.cpp | 22 +++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 57 ++++++--- .../MergeTree/MergeTreeDataSelectExecutor.h | 2 +- src/Storages/SelectQueryInfo.h | 4 + src/Storages/StorageMerge.cpp | 8 +- ...> 02379_analyzer_subquery_depth.reference} | 0 ....sql => 02379_analyzer_subquery_depth.sql} | 0 .../02380_analyzer_join_sample.reference | 2 + .../02380_analyzer_join_sample.sql | 29 +++++ .../02381_analyzer_join_final.reference | 2 + .../0_stateless/02381_analyzer_join_final.sql | 33 +++++ 30 files changed, 671 insertions(+), 55 deletions(-) create mode 100644 src/Analyzer/TableExpressionModifiers.cpp create mode 100644 src/Analyzer/TableExpressionModifiers.h rename tests/queries/0_stateless/{02378_analyzer_subquery_depth.reference => 02379_analyzer_subquery_depth.reference} (100%) rename tests/queries/0_stateless/{02378_analyzer_subquery_depth.sql => 02379_analyzer_subquery_depth.sql} (100%) create mode 100644 tests/queries/0_stateless/02380_analyzer_join_sample.reference create mode 100644 tests/queries/0_stateless/02380_analyzer_join_sample.sql create mode 100644 tests/queries/0_stateless/02381_analyzer_join_final.reference create mode 100644 tests/queries/0_stateless/02381_analyzer_join_final.sql diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp index 5275ec8c586..f9128fa87c9 100644 --- a/src/Analyzer/IdentifierNode.cpp +++ b/src/Analyzer/IdentifierNode.cpp @@ -18,11 +18,26 @@ void IdentifierNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_sta buffer << ", alias: " << getAlias(); buffer << ", identifier: " << identifier.getFullName(); + + if (table_expression_modifiers) + { + buffer << ", "; + table_expression_modifiers->dump(buffer); + } } bool IdentifierNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); + + if (table_expression_modifiers && rhs_typed.table_expression_modifiers && table_expression_modifiers != rhs_typed.table_expression_modifiers) + return false; + else if (table_expression_modifiers && !rhs_typed.table_expression_modifiers) + return false; + else if (!table_expression_modifiers && rhs_typed.table_expression_modifiers) + return false; + + return identifier == rhs_typed.identifier; } @@ -31,6 +46,9 @@ void IdentifierNode::updateTreeHashImpl(HashState & state) const const auto & identifier_name = identifier.getFullName(); state.update(identifier_name.size()); state.update(identifier_name); + + if (table_expression_modifiers) + table_expression_modifiers->updateTreeHash(state); } ASTPtr IdentifierNode::toASTImpl() const diff --git a/src/Analyzer/IdentifierNode.h b/src/Analyzer/IdentifierNode.h index 06bde3d4e0a..a47fb283b15 100644 --- a/src/Analyzer/IdentifierNode.h +++ b/src/Analyzer/IdentifierNode.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -21,12 +22,34 @@ public: : identifier(std::move(identifier_)) {} + /** Construct identifier node with identifier and table expression modifiers + * when identifier node is part of JOIN TREE. + * + * Example: SELECT * FROM test_table SAMPLE 0.1 OFFSET 0.1 FINAL + */ + explicit IdentifierNode(Identifier identifier_, TableExpressionModifiers table_expression_modifiers_) + : identifier(std::move(identifier_)) + , table_expression_modifiers(std::move(table_expression_modifiers_)) + {} + /// Get identifier const Identifier & getIdentifier() const { return identifier; } + /// Return true if identifier node has table expression modifiers, false otherwise + bool hasTableExpressionModifiers() const + { + return table_expression_modifiers.has_value(); + } + + /// Get table expression modifiers + std::optional getTableExpressionModifiers() const + { + return table_expression_modifiers; + } + QueryTreeNodeType getNodeType() const override { return QueryTreeNodeType::IDENTIFIER; @@ -50,6 +73,7 @@ protected: private: Identifier identifier; + std::optional table_expression_modifiers; }; } diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 2cf0cd72ffa..9614ea6c04a 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -101,6 +101,8 @@ namespace ErrorCodes extern const int ILLEGAL_AGGREGATION; extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; + extern const int ILLEGAL_FINAL; + extern const int SAMPLING_NOT_SUPPORTED; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -190,7 +192,6 @@ namespace ErrorCodes * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. * TODO: JOIN support SELF JOIN with MergeTree. JOIN support matchers. * TODO: WINDOW functions - * TODO: Table expression modifiers final, sample_size, sample_offset * TODO: Support function identifier resolve from parent query scope, if lambda in parent scope does not capture any columns. */ @@ -840,6 +841,8 @@ private: static void validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope); + static void validateTableExpressionModifiers(QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + /// Resolve identifier functions QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier); @@ -1086,6 +1089,66 @@ void QueryAnalyzer::validateLimitOffsetExpression(QueryTreeNodePtr & expression_ expression_description); } +void QueryAnalyzer::validateTableExpressionModifiers(QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) +{ + auto * table_node = table_expression_node->as(); + auto * table_function_node = table_expression_node->as(); + auto * query_node = table_expression_node->as(); + auto * union_node = table_expression_node->as(); + + if (!table_node && !table_function_node && !query_node && !union_node) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected table expression. Expected table, table function, query or union node. Actual {}", + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + if (query_node || union_node) + { + auto table_expression_modifiers = query_node ? query_node->getTableExpressionModifiers() : union_node->getTableExpressionModifiers(); + + if (table_expression_modifiers.has_value()) + { + String table_expression_modifiers_error_message; + + if (table_expression_modifiers->hasFinal()) + { + table_expression_modifiers_error_message += "FINAL"; + + if (table_expression_modifiers->hasSampleSizeRatio()) + table_expression_modifiers_error_message += ", SAMPLE"; + } + else if (table_expression_modifiers->hasSampleSizeRatio()) + { + table_expression_modifiers_error_message += "SAMPLE"; + } + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Table expresion modifiers {} are not supported for subquery {}. In scope {}", + table_expression_modifiers_error_message, + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + } + else if (table_node || table_function_node) + { + auto table_expression_modifiers = table_node ? table_node->getTableExpressionModifiers() : table_function_node->getTableExpressionModifiers(); + + if (table_expression_modifiers.has_value()) + { + const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); + if (table_expression_modifiers->hasFinal() && !storage->supportsFinal()) + throw Exception(ErrorCodes::ILLEGAL_FINAL, + "Storage {} doesn't support FINAL", + storage->getName()); + + if (table_expression_modifiers->hasSampleSizeRatio() && !storage->supportsSampling()) + throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, + "Storage {} doesn't support sampling", + storage->getStorageID().getFullNameNotQuoted()); + } + } +} + /// Resolve identifier functions implementation /// Try resolve table identifier from database catalog @@ -2293,7 +2356,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( throw Exception(ErrorCodes::LOGICAL_ERROR, "Unqualified matcher {} resolve unexpected table expression. In scope {}", matcher_node_typed.formatASTForErrorMessage(), - scope_query_node->getJoinTree()->formatASTForErrorMessage()); + scope_query_node->formatASTForErrorMessage()); } for (auto & table_expression_column : table_expression_columns) @@ -3739,10 +3802,38 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod scope.scope_node->formatASTForErrorMessage()); resolved_identifier = resolved_identifier->clone(); - bool resolved_as_cte = resolved_identifier->as() && resolved_identifier->as()->isCTE(); - if (resolved_as_cte) - resolved_identifier->as().setIsCTE(false); + auto table_expression_modifiers = from_table_identifier.getTableExpressionModifiers(); + + if (auto * resolved_identifier_query_node = resolved_identifier->as()) + { + resolved_identifier_query_node->setIsCTE(false); + if (table_expression_modifiers.has_value()) + resolved_identifier_query_node->setTableExpressionModifiers(*table_expression_modifiers); + } + else if (auto * resolved_identifier_union_node = resolved_identifier->as()) + { + resolved_identifier_union_node->setIsCTE(false); + if (table_expression_modifiers.has_value()) + resolved_identifier_union_node->setTableExpressionModifiers(*table_expression_modifiers); + } + else if (auto * resolved_identifier_table_node = resolved_identifier->as()) + { + if (table_expression_modifiers.has_value()) + resolved_identifier_table_node->setTableExpressionModifiers(*table_expression_modifiers); + } + else if (auto * resolved_identifier_table_function_node = resolved_identifier->as();) + { + if (table_expression_modifiers.has_value()) + resolved_identifier_table_function_node->setTableExpressionModifiers(*table_expression_modifiers); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Identifier in JOIN TREE {} resolve unexpected table expression. In scope {}", + from_table_identifier.getIdentifier().getFullName(), + scope.scope_node->formatASTForErrorMessage()); + } auto current_join_tree_node_alias = current_join_tree_node->getAlias(); resolved_identifier->setAlias(current_join_tree_node_alias); @@ -3851,7 +3942,7 @@ void QueryAnalyzer::initializeTableExpressionColumns(QueryTreeNodePtr & table_ex { const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); - auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); + auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns().withVirtuals()); const auto & columns_description = storage_snapshot->metadata->getColumns(); std::vector> alias_columns_to_resolve; @@ -3953,17 +4044,17 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, switch (from_node_type) { case QueryTreeNodeType::QUERY: - { - IdentifierResolveScope subquery_scope(join_tree_node, &scope); - subquery_scope.subquery_depth = scope.subquery_depth + 1; - resolveQuery(join_tree_node, subquery_scope); - break; - } + [[fallthrough]]; case QueryTreeNodeType::UNION: { IdentifierResolveScope subquery_scope(join_tree_node, &scope); subquery_scope.subquery_depth = scope.subquery_depth + 1; - resolveUnion(join_tree_node, subquery_scope); + + if (from_node_type == QueryTreeNodeType::QUERY) + resolveQuery(join_tree_node, subquery_scope); + else if (from_node_type == QueryTreeNodeType::UNION) + resolveUnion(join_tree_node, subquery_scope); + break; } case QueryTreeNodeType::TABLE_FUNCTION: @@ -4173,7 +4264,10 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, join_tree_node_type == QueryTreeNodeType::UNION || join_tree_node_type == QueryTreeNodeType::TABLE || join_tree_node_type == QueryTreeNodeType::TABLE_FUNCTION) + { + validateTableExpressionModifiers(join_tree_node, scope); initializeTableExpressionColumns(join_tree_node, scope); + } add_table_expression_alias_into_scope(join_tree_node); scope.table_expressions_in_resolve_process.erase(join_tree_node.get()); diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index 5e098783fd8..4ae9db8333f 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -38,6 +38,7 @@ namespace DB * No GROUPING function if there is no GROUP BY. * No aggregate functions in WHERE, PREWHERE and inside another aggregate functions. * GROUP BY modifiers CUBE, ROLLUP, GROUPING SETS and WITH TOTALS. + * Table expression modifiers are disabled for subqueries in JOIN TREE. * * 10. Special functions handling: * Function `untuple` is handled properly. diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index e857b35acd9..5c27b01081d 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -145,6 +145,12 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << ", constant_value_type: " << constant_value->getType()->getName(); } + if (table_expression_modifiers) + { + buffer << ", "; + table_expression_modifiers->dump(buffer); + } + if (hasWith()) { buffer << '\n' << std::string(indent + 2, ' ') << "WITH\n"; @@ -246,6 +252,7 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); + if (constant_value && rhs_typed.constant_value && *constant_value != *rhs_typed.constant_value) return false; else if (constant_value && !rhs_typed.constant_value) @@ -253,6 +260,13 @@ bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const else if (!constant_value && rhs_typed.constant_value) return false; + if (table_expression_modifiers && rhs_typed.table_expression_modifiers && table_expression_modifiers != rhs_typed.table_expression_modifiers) + return false; + else if (table_expression_modifiers && !rhs_typed.table_expression_modifiers) + return false; + else if (!table_expression_modifiers && rhs_typed.table_expression_modifiers) + return false; + return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name && @@ -289,6 +303,9 @@ void QueryNode::updateTreeHashImpl(HashState & state) const state.update(constant_value_type_name.size()); state.update(constant_value_type_name); } + + if (table_expression_modifiers) + table_expression_modifiers->updateTreeHash(state); } ASTPtr QueryNode::toASTImpl() const @@ -380,6 +397,7 @@ QueryTreeNodePtr QueryNode::cloneImpl() const result_query_node->cte_name = cte_name; result_query_node->projection_columns = projection_columns; result_query_node->constant_value = constant_value; + result_query_node->table_expression_modifiers = table_expression_modifiers; return result_query_node; } diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 4dc893fcfa7..76769779f3b 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -110,6 +111,24 @@ public: return is_group_by_with_grouping_sets; } + /// Return true if query node has table expression modifiers, false otherwise + bool hasTableExpressionModifiers() const + { + return table_expression_modifiers.has_value(); + } + + /// Get table expression modifiers + std::optional getTableExpressionModifiers() const + { + return table_expression_modifiers; + } + + /// Set table expression modifiers + void setTableExpressionModifiers(TableExpressionModifiers table_expression_modifiers_value) + { + table_expression_modifiers = std::move(table_expression_modifiers_value); + } + bool hasWith() const { return !getWith().getNodes().empty(); @@ -420,6 +439,7 @@ private: std::string cte_name; NamesAndTypes projection_columns; ConstantValuePtr constant_value; + std::optional table_expression_modifiers; static constexpr size_t with_child_index = 0; static constexpr size_t projection_child_index = 1; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 27861fd38d8..a845f453dbf 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include @@ -204,9 +205,10 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q current_query_tree->setIsGroupByWithCube(select_query_typed.group_by_with_cube); current_query_tree->setIsGroupByWithRollup(select_query_typed.group_by_with_rollup); current_query_tree->setIsGroupByWithGroupingSets(select_query_typed.group_by_with_grouping_sets); - current_query_tree->getJoinTree() = buildJoinTree(select_query_typed.tables()); current_query_tree->setOriginalAST(select_query); + current_query_tree->getJoinTree() = buildJoinTree(select_query_typed.tables()); + auto select_with_list = select_query_typed.with(); if (select_with_list) current_query_tree->getWithNode() = buildExpressionList(select_with_list); @@ -542,17 +544,44 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select if (table_element.table_expression) { auto & table_expression = table_element.table_expression->as(); + std::optional table_expression_modifiers; + + if (table_expression.final || table_expression.sample_size) + { + bool has_final = table_expression.final; + std::optional sample_size_ratio; + std::optional sample_offset_ratio; + + if (table_expression.sample_size) + { + auto & ast_sample_size_ratio = table_expression.sample_size->as(); + sample_size_ratio = ast_sample_size_ratio.ratio; + + if (table_expression.sample_offset) + { + auto & ast_sample_offset_ratio = table_expression.sample_offset->as(); + sample_offset_ratio = ast_sample_offset_ratio.ratio; + } + } + + table_expression_modifiers = TableExpressionModifiers(has_final, sample_size_ratio, sample_offset_ratio); + } if (table_expression.database_and_table_name) { auto & table_identifier_typed = table_expression.database_and_table_name->as(); auto storage_identifier = Identifier(table_identifier_typed.name_parts); - auto node = std::make_shared(storage_identifier); + QueryTreeNodePtr table_identifier_node; - node->setAlias(table_identifier_typed.tryGetAlias()); - node->setOriginalAST(table_element.table_expression); + if (table_expression_modifiers) + table_identifier_node = std::make_shared(storage_identifier, *table_expression_modifiers); + else + table_identifier_node = std::make_shared(storage_identifier); - table_expressions.push_back(std::move(node)); + table_identifier_node->setAlias(table_identifier_typed.tryGetAlias()); + table_identifier_node->setOriginalAST(table_element.table_expression); + + table_expressions.push_back(std::move(table_identifier_node)); } else if (table_expression.subquery) { @@ -560,10 +589,21 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select const auto & select_with_union_query = subquery_expression.children[0]; auto node = buildSelectWithUnionExpression(select_with_union_query, true /*is_subquery*/, {} /*cte_name*/); - node->setAlias(subquery_expression.tryGetAlias()); node->setOriginalAST(select_with_union_query); + if (table_expression_modifiers) + { + if (auto * query_node = node->as()) + query_node->setTableExpressionModifiers(*table_expression_modifiers); + else if (auto * union_node = node->as()) + union_node->setTableExpressionModifiers(*table_expression_modifiers); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected table expression subquery node. Expected union or query. Actual {}", + node->formatASTForErrorMessage()); + } + table_expressions.push_back(std::move(node)); } else if (table_expression.table_function) @@ -584,6 +624,8 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select } } + if (table_expression_modifiers) + node->setTableExpressionModifiers(*table_expression_modifiers); node->setAlias(table_function_expression.tryGetAlias()); node->setOriginalAST(table_expression.table_function); diff --git a/src/Analyzer/TableExpressionModifiers.cpp b/src/Analyzer/TableExpressionModifiers.cpp new file mode 100644 index 00000000000..79b5a8dba41 --- /dev/null +++ b/src/Analyzer/TableExpressionModifiers.cpp @@ -0,0 +1,42 @@ +#include + +#include + +#include +#include +#include + +namespace DB +{ + +void TableExpressionModifiers::dump(WriteBuffer & buffer) const +{ + buffer << "final: " << has_final; + + if (sample_size_ratio) + buffer << ", sample_size: " << ASTSampleRatio::toString(*sample_size_ratio); + + if (sample_offset_ratio) + buffer << ", sample_offset: " << ASTSampleRatio::toString(*sample_offset_ratio); +} + +void TableExpressionModifiers::updateTreeHash(SipHash & hash_state) const +{ + hash_state.update(has_final); + hash_state.update(sample_size_ratio.has_value()); + hash_state.update(sample_offset_ratio.has_value()); + + if (sample_size_ratio.has_value()) + { + hash_state.update(sample_size_ratio->numerator); + hash_state.update(sample_size_ratio->denominator); + } + + if (sample_offset_ratio.has_value()) + { + hash_state.update(sample_offset_ratio->numerator); + hash_state.update(sample_offset_ratio->denominator); + } +} + +} diff --git a/src/Analyzer/TableExpressionModifiers.h b/src/Analyzer/TableExpressionModifiers.h new file mode 100644 index 00000000000..c7a65d4c2b5 --- /dev/null +++ b/src/Analyzer/TableExpressionModifiers.h @@ -0,0 +1,71 @@ +#pragma once + +#include + +namespace DB +{ + +/** Modifiers that can be used for table, table function and subquery in JOIN TREE. + * + * Example: SELECT * FROM test_table SAMPLE 0.1 OFFSET 0.1 FINAL + */ +class TableExpressionModifiers +{ +public: + using Rational = ASTSampleRatio::Rational; + + TableExpressionModifiers(bool has_final_, + std::optional sample_size_ratio_, + std::optional sample_offset_ratio_) + : has_final(has_final_) + , sample_size_ratio(sample_size_ratio_) + , sample_offset_ratio(sample_offset_ratio_) + {} + + bool hasFinal() const + { + return has_final; + } + + bool hasSampleSizeRatio() const + { + return sample_size_ratio.has_value(); + } + + std::optional getSampleSizeRatio() const + { + return sample_size_ratio; + } + + bool hasSampleOffsetRatio() const + { + return sample_offset_ratio.has_value(); + } + + std::optional getSampleOffsetRatio() const + { + return sample_offset_ratio; + } + + void dump(WriteBuffer & buffer) const; + + void updateTreeHash(SipHash & hash_state) const; + +private: + bool has_final; + std::optional sample_size_ratio; + std::optional sample_offset_ratio; +}; + +inline bool operator==(const TableExpressionModifiers & lhs, const TableExpressionModifiers & rhs) +{ + lhs.getSampleOffsetRatio(); + return lhs.hasFinal() == rhs.hasFinal() && lhs.getSampleSizeRatio() == rhs.getSampleSizeRatio() && lhs.getSampleOffsetRatio() == rhs.getSampleOffsetRatio(); +} + +inline bool operator!=(const TableExpressionModifiers & lhs, const TableExpressionModifiers & rhs) +{ + return !(lhs == rhs); +} + +} diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index 7063d6ad149..0e6c1f93502 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -74,6 +74,12 @@ void TableFunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_ buffer << ", table_function_name: " << table_function_name; + if (table_expression_modifiers) + { + buffer << ", "; + table_expression_modifiers->dump(buffer); + } + const auto & arguments = getArguments(); if (!arguments.getNodes().empty()) { @@ -91,6 +97,13 @@ bool TableFunctionNode::isEqualImpl(const IQueryTreeNode & rhs) const if (storage && rhs_typed.storage) return storage_id == rhs_typed.storage_id; + if (table_expression_modifiers && rhs_typed.table_expression_modifiers && table_expression_modifiers != rhs_typed.table_expression_modifiers) + return false; + else if (table_expression_modifiers && !rhs_typed.table_expression_modifiers) + return false; + else if (!table_expression_modifiers && rhs_typed.table_expression_modifiers) + return false; + return true; } @@ -105,6 +118,9 @@ void TableFunctionNode::updateTreeHashImpl(HashState & state) const state.update(full_name.size()); state.update(full_name); } + + if (table_expression_modifiers) + table_expression_modifiers->updateTreeHash(state); } ASTPtr TableFunctionNode::toASTImpl() const @@ -130,6 +146,7 @@ QueryTreeNodePtr TableFunctionNode::cloneImpl() const result->storage = storage; result->storage_id = storage_id; result->storage_snapshot = storage_snapshot; + result->table_expression_modifiers = table_expression_modifiers; return result; } diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index 794967870f0..0b02842f37c 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -9,7 +9,7 @@ #include #include - +#include namespace DB { @@ -98,6 +98,25 @@ public: /// Get storage snapshot, throws exception if function node is not resolved const StorageSnapshotPtr & getStorageSnapshot() const; + /// Return true if table function node has table expression modifiers, false otherwise + bool hasTableExpressionModifiers() const + { + return table_expression_modifiers.has_value(); + } + + /// Get table expression modifiers + std::optional getTableExpressionModifiers() const + { + return table_expression_modifiers; + } + + /// Set table expression modifiers + void setTableExpressionModifiers(TableExpressionModifiers table_expression_modifiers_value) + { + table_expression_modifiers = std::move(table_expression_modifiers_value); + } + + QueryTreeNodeType getNodeType() const override { return QueryTreeNodeType::TABLE_FUNCTION; @@ -124,6 +143,7 @@ private: StoragePtr storage; StorageID storage_id; StorageSnapshotPtr storage_snapshot; + std::optional table_expression_modifiers; }; } diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index 768e6f0e6a3..e128987d179 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -31,11 +31,25 @@ void TableNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << ", alias: " << getAlias(); buffer << ", table_name: " << storage_id.getFullNameNotQuoted(); + + if (table_expression_modifiers) + { + buffer << ", "; + table_expression_modifiers->dump(buffer); + } } bool TableNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); + + if (table_expression_modifiers && rhs_typed.table_expression_modifiers && table_expression_modifiers != rhs_typed.table_expression_modifiers) + return false; + else if (table_expression_modifiers && !rhs_typed.table_expression_modifiers) + return false; + else if (!table_expression_modifiers && rhs_typed.table_expression_modifiers) + return false; + return storage_id == rhs_typed.storage_id; } @@ -44,6 +58,9 @@ void TableNode::updateTreeHashImpl(HashState & state) const auto full_name = storage_id.getFullNameNotQuoted(); state.update(full_name.size()); state.update(full_name); + + if (table_expression_modifiers) + table_expression_modifiers->updateTreeHash(state); } String TableNode::getName() const @@ -64,6 +81,7 @@ QueryTreeNodePtr TableNode::cloneImpl() const result_table_node->storage_id = storage_id; result_table_node->storage_lock = storage_lock; result_table_node->storage_snapshot = storage_snapshot; + result_table_node->table_expression_modifiers = table_expression_modifiers; return result_table_node; } diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index 0a528b4db67..b06166c5b0d 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -8,6 +8,7 @@ #include #include +#include namespace DB { @@ -61,6 +62,24 @@ public: return std::move(storage_lock); } + /// Return true if table node has table expression modifiers, false otherwise + bool hasTableExpressionModifiers() const + { + return table_expression_modifiers.has_value(); + } + + /// Get table expression modifiers + std::optional getTableExpressionModifiers() const + { + return table_expression_modifiers; + } + + /// Set table expression modifiers + void setTableExpressionModifiers(TableExpressionModifiers table_expression_modifiers_value) + { + table_expression_modifiers = std::move(table_expression_modifiers_value); + } + QueryTreeNodeType getNodeType() const override { return QueryTreeNodeType::TABLE; @@ -86,6 +105,7 @@ private: StorageID storage_id; TableLockHolder storage_lock; StorageSnapshotPtr storage_snapshot; + std::optional table_expression_modifiers; }; } diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 8fbd3bb193e..8ff26664711 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -127,6 +127,12 @@ void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << ", constant_value_type: " << constant_value->getType()->getName(); } + if (table_expression_modifiers) + { + buffer << ", "; + table_expression_modifiers->dump(buffer); + } + buffer << ", union_mode: "; if (union_mode == SelectUnionMode::ALL || union_mode == SelectUnionMode::DISTINCT) @@ -165,6 +171,13 @@ bool UnionNode::isEqualImpl(const IQueryTreeNode & rhs) const else if (!constant_value && rhs_typed.constant_value) return false; + if (table_expression_modifiers && rhs_typed.table_expression_modifiers && table_expression_modifiers != rhs_typed.table_expression_modifiers) + return false; + else if (table_expression_modifiers && !rhs_typed.table_expression_modifiers) + return false; + else if (!table_expression_modifiers && rhs_typed.table_expression_modifiers) + return false; + return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name; } @@ -186,6 +199,9 @@ void UnionNode::updateTreeHashImpl(HashState & state) const state.update(constant_value_type_name.size()); state.update(constant_value_type_name); } + + if (table_expression_modifiers) + table_expression_modifiers->updateTreeHash(state); } ASTPtr UnionNode::toASTImpl() const @@ -211,6 +227,7 @@ QueryTreeNodePtr UnionNode::cloneImpl() const result_query_node->union_modes = union_modes; result_query_node->union_modes_set = union_modes_set; result_query_node->constant_value = constant_value; + result_query_node->table_expression_modifiers = table_expression_modifiers; return result_query_node; } diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h index f153fda575b..1ca90b91bd1 100644 --- a/src/Analyzer/UnionNode.h +++ b/src/Analyzer/UnionNode.h @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -93,6 +94,25 @@ public: return children[queries_child_index]->as(); } + /// Return true if union node has table expression modifiers, false otherwise + bool hasTableExpressionModifiers() const + { + return table_expression_modifiers.has_value(); + } + + /// Get table expression modifiers + std::optional getTableExpressionModifiers() const + { + return table_expression_modifiers; + } + + /// Set table expression modifiers + void setTableExpressionModifiers(TableExpressionModifiers table_expression_modifiers_value) + { + table_expression_modifiers = std::move(table_expression_modifiers_value); + } + + /// Compute union projection NamesAndTypes computeProjectionColumns() const; @@ -141,6 +161,7 @@ private: SelectUnionModes union_modes; SelectUnionModesSet union_modes_set; ConstantValuePtr constant_value; + std::optional table_expression_modifiers; static constexpr size_t queries_child_index = 0; static constexpr size_t children_size = queries_child_index + 1; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index e8d3852cf76..9d1a653e4a7 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -10,6 +10,9 @@ #include #include #include +#include +#include +#include namespace DB { @@ -74,14 +77,52 @@ static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expre auto result_table_expression = std::make_shared(); result_table_expression->children.push_back(table_expression_node_ast); + std::optional table_expression_modifiers; + if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) + { + if (auto * query_node = table_expression_node->as()) + table_expression_modifiers = query_node->getTableExpressionModifiers(); + else if (auto * union_node = table_expression_node->as()) + table_expression_modifiers = union_node->getTableExpressionModifiers(); + result_table_expression->subquery = result_table_expression->children.back(); + } else if (node_type == QueryTreeNodeType::TABLE || node_type == QueryTreeNodeType::IDENTIFIER) + { + if (auto * table_node = table_expression_node->as()) + table_expression_modifiers = table_node->getTableExpressionModifiers(); + else if (auto * identifier_node = table_expression_node->as()) + table_expression_modifiers = identifier_node->getTableExpressionModifiers(); + result_table_expression->database_and_table_name = result_table_expression->children.back(); + } else if (node_type == QueryTreeNodeType::TABLE_FUNCTION) + { + if (auto * table_function_node = table_expression_node->as()) + table_expression_modifiers = table_function_node->getTableExpressionModifiers(); + result_table_expression->table_function = result_table_expression->children.back(); + } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected identiifer, table, query, union or table function. Actual {}", table_expression_node->formatASTForErrorMessage()); + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected identifier, table, query, union or table function. Actual {}", + table_expression_node->formatASTForErrorMessage()); + } + + if (table_expression_modifiers) + { + result_table_expression->final = table_expression_modifiers->hasFinal(); + + auto sample_size_ratio = table_expression_modifiers->getSampleSizeRatio(); + if (sample_size_ratio.has_value()) + result_table_expression->sample_size = std::make_shared(*sample_size_ratio); + + auto sample_offset_ratio = table_expression_modifiers->getSampleOffsetRatio(); + if (sample_offset_ratio.has_value()) + result_table_expression->sample_offset = std::make_shared(*sample_offset_ratio); + } return result_table_expression; } diff --git a/src/Parsers/ASTSampleRatio.h b/src/Parsers/ASTSampleRatio.h index a3e70b7dab7..220f938335b 100644 --- a/src/Parsers/ASTSampleRatio.h +++ b/src/Parsers/ASTSampleRatio.h @@ -34,4 +34,14 @@ public: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; }; +inline bool operator==(const ASTSampleRatio::Rational & lhs, const ASTSampleRatio::Rational & rhs) +{ + return lhs.numerator == rhs.numerator && lhs.denominator == rhs.denominator; +} + +inline bool operator!=(const ASTSampleRatio::Rational & lhs, const ASTSampleRatio::Rational & rhs) +{ + return !(lhs == rhs); +} + } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 565dcb1a6b3..65c6e6110e5 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -324,7 +324,7 @@ QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, PlannerContextPtr & planner_context); QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, - SelectQueryInfo & table_expression_query_info, + SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, PlannerContextPtr & planner_context) { @@ -348,6 +348,12 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + auto table_expression_query_info = select_query_info; + if (table_node) + table_expression_query_info.table_expression_modifiers = table_node->getTableExpressionModifiers(); + else + table_expression_query_info.table_expression_modifiers = table_function_node->getTableExpressionModifiers(); + auto from_stage = storage->getQueryProcessingStage(planner_context->getQueryContext(), select_query_options.to_stage, storage_snapshot, table_expression_query_info); const auto & columns_names = table_expression_columns.getColumnsNames(); Names column_names(columns_names.begin(), columns_names.end()); @@ -745,8 +751,7 @@ QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, [[fallthrough]]; case QueryTreeNodeType::TABLE_FUNCTION: { - SelectQueryInfo table_expression_query_info = select_query_info; - return buildQueryPlanForTableExpression(join_tree_node, table_expression_query_info, select_query_options, planner_context); + return buildQueryPlanForTableExpression(join_tree_node, select_query_info, select_query_options, planner_context); } case QueryTreeNodeType::JOIN: { diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 59f9668d3a8..62b6eddf6ce 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -945,7 +945,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( result.index_stats); result.sampling = MergeTreeDataSelectExecutor::getSampling( - select, + query_info, metadata_snapshot->getColumns().getAllPhysical(), parts, *key_condition, @@ -965,7 +965,13 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( auto reader_settings = getMergeTreeReaderSettings(context, query_info); bool use_skip_indexes = settings.use_skip_indexes; - if (select.final() && !settings.use_skip_indexes_if_final) + bool final = false; + if (query_info.table_expression_modifiers) + final = query_info.table_expression_modifiers->hasFinal(); + else + final = select.final(); + + if (final && !settings.use_skip_indexes_if_final) use_skip_indexes = false; result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( @@ -1097,7 +1103,13 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons 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) + bool final = false; + if (query_info.table_expression_modifiers) + final = query_info.table_expression_modifiers->hasFinal(); + else + final = select.final(); + + if (!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. @@ -1112,7 +1124,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons const auto & input_order_info = query_info.getInputOrderInfo(); - if (select.final()) + if (final) { /// Add columns needed to calculate the sorting expression and the sign. std::vector add_columns = metadata_for_reading->getColumnsRequiredForSortingKey(); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index d7c33c8663b..4e73323ee89 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -848,9 +848,11 @@ Block KeyCondition::getBlockWithConstants( { DataTypeUInt8().createColumnConstWithDefaultValue(1), std::make_shared(), "_dummy" } }; - const auto expr_for_constant_folding = ExpressionAnalyzer(query, syntax_analyzer_result, context).getConstActions(); - - expr_for_constant_folding->execute(result); + if (syntax_analyzer_result) + { + const auto expr_for_constant_folding = ExpressionAnalyzer(query, syntax_analyzer_result, context).getConstActions(); + expr_for_constant_folding->execute(result); + } return result; } @@ -892,8 +894,11 @@ KeyCondition::KeyCondition( */ Block block_with_constants = getBlockWithConstants(query, syntax_analyzer_result, context); - for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source) - array_joined_columns.insert(name); + if (syntax_analyzer_result) + { + for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source) + array_joined_columns.insert(name); + } const ASTSelectQuery & select = query->as(); @@ -964,8 +969,11 @@ KeyCondition::KeyCondition( key_columns[name] = i; } - for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source) - array_joined_columns.insert(name); + if (syntax_analyzer_result) + { + for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source) + array_joined_columns.insert(name); + } if (!dag_nodes.nodes.empty()) { diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 5b3497bf926..322790b6b14 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -107,14 +107,12 @@ static std::string toString(const RelativeSize & x) } /// Converts sample size to an approximate number of rows (ex. `SAMPLE 1000000`) to relative value (ex. `SAMPLE 0.1`). -static RelativeSize convertAbsoluteSampleSizeToRelative(const ASTPtr & node, size_t approx_total_rows) +static RelativeSize convertAbsoluteSampleSizeToRelative(const ASTSampleRatio::Rational & ratio, size_t approx_total_rows) { if (approx_total_rows == 0) return 1; - const auto & node_sample = node->as(); - - auto absolute_sample_size = node_sample.ratio.numerator / node_sample.ratio.denominator; + auto absolute_sample_size = ratio.numerator / ratio.denominator; return std::min(RelativeSize(1), RelativeSize(absolute_sample_size) / RelativeSize(approx_total_rows)); } @@ -467,7 +465,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( } MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( - const ASTSelectQuery & select, + const SelectQueryInfo & select_query_info, NamesAndTypesList available_real_columns, const MergeTreeData::DataPartsVector & parts, KeyCondition & key_condition, @@ -484,23 +482,46 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( RelativeSize relative_sample_size = 0; RelativeSize relative_sample_offset = 0; - auto select_sample_size = select.sampleSize(); - auto select_sample_offset = select.sampleOffset(); + bool final = false; + std::optional sample_size_ratio; + std::optional sample_offset_ratio; - if (select_sample_size) + if (select_query_info.table_expression_modifiers) { - relative_sample_size.assign( - select_sample_size->as().ratio.numerator, - select_sample_size->as().ratio.denominator); + const auto & table_expression_modifiers = *select_query_info.table_expression_modifiers; + final = table_expression_modifiers.hasFinal(); + + if (table_expression_modifiers.hasSampleSizeRatio()) + sample_size_ratio = table_expression_modifiers.getSampleSizeRatio(); + + if (table_expression_modifiers.hasSampleOffsetRatio()) + sample_offset_ratio = table_expression_modifiers.getSampleSizeRatio(); + } + else + { + auto & select = select_query_info.query->as(); + + final = select.final(); + auto select_sample_size = select.sampleSize(); + auto select_sample_offset = select.sampleOffset(); + + if (select_sample_size) + sample_size_ratio = select_sample_size->as().ratio; + + if (select_sample_offset) + sample_offset_ratio = select_sample_offset->as().ratio; + } + + if (sample_size_ratio) + { + relative_sample_size.assign(sample_size_ratio->numerator, sample_size_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 (sample_offset_ratio) + relative_sample_offset.assign(sample_offset_ratio->numerator, sample_offset_ratio->denominator); if (relative_sample_offset < 0) throw Exception("Negative sample offset", ErrorCodes::ARGUMENT_OUT_OF_BOUND); @@ -513,7 +534,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (relative_sample_size > 1) { - relative_sample_size = convertAbsoluteSampleSizeToRelative(select_sample_size, approx_total_rows); + relative_sample_size = convertAbsoluteSampleSizeToRelative(*sample_size_ratio, approx_total_rows); LOG_DEBUG(log, "Selected relative sample size: {}", toString(relative_sample_size)); } @@ -526,7 +547,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (relative_sample_offset > 1) { - relative_sample_offset = convertAbsoluteSampleSizeToRelative(select_sample_offset, approx_total_rows); + relative_sample_offset = convertAbsoluteSampleSizeToRelative(*sample_offset_ratio, approx_total_rows); LOG_DEBUG(log, "Selected relative sample offset: {}", toString(relative_sample_offset)); } } @@ -660,7 +681,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( /// So, assume that we already have calculated column. ASTPtr sampling_key_ast = metadata_snapshot->getSamplingKeyAST(); - if (select.final()) + if (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. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 66dd7f7e5db..541f6446674 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -201,7 +201,7 @@ public: /// Also, calculate _sample_factor if needed. /// Also, update key condition with selected sampling range. static MergeTreeDataSelectSamplingData getSampling( - const ASTSelectQuery & select, + const SelectQueryInfo & select_query_info, NamesAndTypesList available_real_columns, const MergeTreeData::DataPartsVector & parts, KeyCondition & key_condition, diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index f2835ab4dbf..cbee3365754 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -177,6 +178,9 @@ struct SelectQueryInfo ASTPtr view_query; /// Optimized VIEW query ASTPtr original_query; /// Unmodified query for projection analysis + /// Table expression modifiers for storage + std::optional table_expression_modifiers; + std::shared_ptr storage_limits; /// Cluster for the query. diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c9067148739..dc94b6cee98 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -513,7 +513,13 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( SelectQueryOptions(processed_stage).analyze()).buildQueryPipeline()); } - if (!modified_select.final() && storage->needRewriteQueryWithFinal(real_column_names)) + bool final = false; + if (modified_query_info.table_expression_modifiers) + final = modified_query_info.table_expression_modifiers->hasFinal(); + else + final = modified_select.final(); + + if (!final && storage->needRewriteQueryWithFinal(real_column_names)) { /// NOTE: It may not work correctly in some cases, because query was analyzed without final. /// However, it's needed for MaterializedMySQL and it's unlikely that someone will use it with Merge tables. diff --git a/tests/queries/0_stateless/02378_analyzer_subquery_depth.reference b/tests/queries/0_stateless/02379_analyzer_subquery_depth.reference similarity index 100% rename from tests/queries/0_stateless/02378_analyzer_subquery_depth.reference rename to tests/queries/0_stateless/02379_analyzer_subquery_depth.reference diff --git a/tests/queries/0_stateless/02378_analyzer_subquery_depth.sql b/tests/queries/0_stateless/02379_analyzer_subquery_depth.sql similarity index 100% rename from tests/queries/0_stateless/02378_analyzer_subquery_depth.sql rename to tests/queries/0_stateless/02379_analyzer_subquery_depth.sql diff --git a/tests/queries/0_stateless/02380_analyzer_join_sample.reference b/tests/queries/0_stateless/02380_analyzer_join_sample.reference new file mode 100644 index 00000000000..14d5f58d76a --- /dev/null +++ b/tests/queries/0_stateless/02380_analyzer_join_sample.reference @@ -0,0 +1,2 @@ +0 0 2 2 +1 1 2 2 diff --git a/tests/queries/0_stateless/02380_analyzer_join_sample.sql b/tests/queries/0_stateless/02380_analyzer_join_sample.sql new file mode 100644 index 00000000000..1cf5a4a04df --- /dev/null +++ b/tests/queries/0_stateless/02380_analyzer_join_sample.sql @@ -0,0 +1,29 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table_join_1; +CREATE TABLE test_table_join_1 +( + id UInt64, + value String +) ENGINE=MergeTree +ORDER BY id +SAMPLE BY id; + +INSERT INTO test_table_join_1 VALUES (0, 'Value'), (1, 'Value_1'); + +DROP TABLE IF EXISTS test_table_join_2; +CREATE TABLE test_table_join_2 +( + id UInt64, + value String +) ENGINE=MergeTree +ORDER BY id +SAMPLE BY id; + +INSERT INTO test_table_join_2 VALUES (0, 'Value'), (1, 'Value_1'); + +SELECT t1.id AS t1_id, t2.id AS t2_id, t1._sample_factor AS t1_sample_factor, t2._sample_factor AS t2_sample_factor +FROM test_table_join_1 AS t1 SAMPLE 1/2 INNER JOIN test_table_join_2 AS t2 SAMPLE 1/2 ON t1.id = t2.id; + +DROP TABLE test_table_join_1; +DROP TABLE test_table_join_2; diff --git a/tests/queries/0_stateless/02381_analyzer_join_final.reference b/tests/queries/0_stateless/02381_analyzer_join_final.reference new file mode 100644 index 00000000000..e00d444d142 --- /dev/null +++ b/tests/queries/0_stateless/02381_analyzer_join_final.reference @@ -0,0 +1,2 @@ +0 0 3 1 +1 1 1 3 diff --git a/tests/queries/0_stateless/02381_analyzer_join_final.sql b/tests/queries/0_stateless/02381_analyzer_join_final.sql new file mode 100644 index 00000000000..0881492edeb --- /dev/null +++ b/tests/queries/0_stateless/02381_analyzer_join_final.sql @@ -0,0 +1,33 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table_join_1; +CREATE TABLE test_table_join_1 +( + id UInt64, + value UInt64 +) ENGINE=SummingMergeTree(value) +ORDER BY id +SAMPLE BY id; + +SYSTEM STOP MERGES test_table_join_1; +INSERT INTO test_table_join_1 VALUES (0, 1), (1, 1); +INSERT INTO test_table_join_1 VALUES (0, 2); + +DROP TABLE IF EXISTS test_table_join_2; +CREATE TABLE test_table_join_2 +( + id UInt64, + value UInt64 +) ENGINE=SummingMergeTree(value) +ORDER BY id +SAMPLE BY id; + +SYSTEM STOP MERGES test_table_join_2; +INSERT INTO test_table_join_2 VALUES (0, 1), (1, 1); +INSERT INTO test_table_join_2 VALUES (1, 2); + +SELECT t1.id AS t1_id, t2.id AS t2_id, t1.value AS t1_value, t2.value AS t2_value +FROM test_table_join_1 AS t1 FINAL INNER JOIN test_table_join_2 AS t2 FINAL ON t1.id = t2.id; + +DROP TABLE test_table_join_1; +DROP TABLE test_table_join_2; From 1844673aa86060a8ab7afa3715314e62b68019b7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 5 Sep 2022 11:46:18 +0200 Subject: [PATCH 094/188] Updated tests --- src/Analyzer/QueryAnalysisPass.cpp | 36 ++- src/Analyzer/Utils.cpp | 6 +- .../02378_analyzer_projection_names.reference | 225 +++++++++++++++++- .../02378_analyzer_projection_names.sql | 124 +++++++++- 4 files changed, 375 insertions(+), 16 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 9614ea6c04a..e8da63e0614 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -1222,10 +1222,14 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(cons return {}; } - if (identifier_lookup.isExpressionLookup() && it->second->getNodeType() != QueryTreeNodeType::COLUMN && it->second->getNodeType() != QueryTreeNodeType::CONSTANT - && it->second->getNodeType() != QueryTreeNodeType::FUNCTION && it->second->getNodeType() != QueryTreeNodeType::QUERY) + auto node_type = it->second->getNodeType(); + if (identifier_lookup.isExpressionLookup() && node_type != QueryTreeNodeType::COLUMN && node_type != QueryTreeNodeType::CONSTANT + && node_type != QueryTreeNodeType::FUNCTION && node_type != QueryTreeNodeType::QUERY && node_type != QueryTreeNodeType::UNION) return {}; - else if (identifier_lookup.isTableLookup() && it->second->getNodeType() != QueryTreeNodeType::TABLE && it->second->getNodeType() != QueryTreeNodeType::QUERY) + else if (identifier_lookup.isTableLookup() && node_type != QueryTreeNodeType::TABLE && node_type != QueryTreeNodeType::TABLE_FUNCTION && + node_type != QueryTreeNodeType::QUERY && node_type != QueryTreeNodeType::UNION) + return {}; + else if (identifier_lookup.isFunctionLookup() && node_type != QueryTreeNodeType::LAMBDA) return {}; if (!resolve_full_identifier && identifier_lookup.identifier.isCompound() && identifier_lookup.isExpressionLookup()) @@ -1613,6 +1617,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id IdentifierLookup column_identifier_lookup {qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION}; bool can_bind_identifier_to_table_expression = tryBindIdentifierToTableExpression(column_identifier_lookup, table_expression_to_check, scope); + if (can_bind_identifier_to_table_expression) { can_remove_qualificator = false; @@ -1665,6 +1670,21 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) { const auto & from_join_node = table_expression_node->as(); + auto left_resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_join_node.getLeftTableExpression(), scope); + auto right_resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_join_node.getRightTableExpression(), scope); + + if (!identifier_lookup.isExpressionLookup()) + { + if (left_resolved_identifier && right_resolved_identifier) + throw Exception(ErrorCodes::AMBIGUOUS_IDENTIFIER, + "JOIN {} ambigious identifier {}. In scope {}", + table_expression_node->formatASTForErrorMessage(), + identifier_lookup.dump(), + scope.scope_node->formatASTForErrorMessage()); + + return left_resolved_identifier ? left_resolved_identifier : right_resolved_identifier; + } + bool join_node_in_resolve_process = scope.table_expressions_in_resolve_process.contains(table_expression_node.get()); std::unordered_map join_using_column_name_to_column_node; @@ -1680,9 +1700,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo } } - auto left_resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_join_node.getLeftTableExpression(), scope); - auto right_resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_join_node.getRightTableExpression(), scope); - std::optional resolved_side; QueryTreeNodePtr resolved_identifier; @@ -1720,7 +1737,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo throw Exception(ErrorCodes::AMBIGUOUS_IDENTIFIER, "JOIN {} ambigious identifier {}. In scope {}", table_expression_node->formatASTForErrorMessage(), - identifier_lookup.identifier.getFullName(), + identifier_lookup.dump(), scope.scope_node->formatASTForErrorMessage()); } } @@ -3559,10 +3576,11 @@ String QueryAnalyzer::calculateProjectionNodeDisplayName(QueryTreeNodePtr & node if (resolved_identifier_result.resolved_identifier && resolved_identifier_result.isResolvedFromJoinTree()) { - projection_name_from_scope = try_to_get_projection_name_from_scope(node); + projection_name_from_scope = try_to_get_projection_name_from_scope(resolved_identifier_result.resolved_identifier); if (!projection_name_from_scope.empty()) return projection_name_from_scope; + if (auto * column_node = resolved_identifier_result.resolved_identifier->as()) return column_node->getColumnName(); } @@ -3822,7 +3840,7 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod if (table_expression_modifiers.has_value()) resolved_identifier_table_node->setTableExpressionModifiers(*table_expression_modifiers); } - else if (auto * resolved_identifier_table_function_node = resolved_identifier->as();) + else if (auto * resolved_identifier_table_function_node = resolved_identifier->as()) { if (table_expression_modifiers.has_value()) resolved_identifier_table_function_node->setTableExpressionModifiers(*table_expression_modifiers); diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 9d1a653e4a7..37e391d0b11 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -201,14 +201,14 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node) case QueryTreeNodeType::ARRAY_JOIN: { auto & array_join_node = node_to_process->as(); - nodes_to_process.push_back(array_join_node.getTableExpression()); + nodes_to_process.push_front(array_join_node.getTableExpression()); break; } case QueryTreeNodeType::JOIN: { auto & join_node = node_to_process->as(); - nodes_to_process.push_back(join_node.getLeftTableExpression()); - nodes_to_process.push_back(join_node.getRightTableExpression()); + nodes_to_process.push_front(join_node.getRightTableExpression()); + nodes_to_process.push_front(join_node.getLeftTableExpression()); break; } default: diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.reference b/tests/queries/0_stateless/02378_analyzer_projection_names.reference index f37fb030ed4..179b125cc08 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.reference +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.reference @@ -253,5 +253,226 @@ arrayMap(a, [1, 2, 3]) Array(Nullable(UInt8)) arrayMap(lambda(tuple(x), _subquery_3), [1, 2, 3]) Array(Nullable(UInt8)) SELECT '--'; -- -SELECT (SELECT 1 AS a, 2 AS b) AS c, c.a, c.b; -(1,2) 1 2 +DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b) AS c, c.a, c.b); +c Tuple(a UInt8, b UInt8) +c.a UInt8 +c.b UInt8 +SELECT 'Joins'; +Joins +DESCRIBE (SELECT * FROM test_table_join_1, test_table_join_2); +test_table_join_1.id UInt64 +test_table_join_1.value String +value_join_1 String +test_table_join_2.id UInt64 +test_table_join_2.value String +value_join_2 String +SELECT '--'; +-- +DESCRIBE (SELECT * FROM test_table_join_1 AS t1, test_table_join_2 AS t2); +t1.id UInt64 +t1.value String +value_join_1 String +t2.id UInt64 +t2.value String +value_join_2 String +SELECT '--'; +-- +DESCRIBE (SELECT * APPLY toString FROM test_table_join_1 AS t1, test_table_join_2 AS t2); +toString(t1.id) String +toString(t1.value) String +toString(value_join_1) String +toString(t2.id) String +toString(t2.value) String +toString(value_join_2) String +SELECT '--'; +-- +DESCRIBE (SELECT * APPLY x -> toString(x) FROM test_table_join_1 AS t1, test_table_join_2 AS t2); +toString(t1.id) String +toString(t1.value) String +toString(value_join_1) String +toString(t2.id) String +toString(t2.value) String +toString(value_join_2) String +SELECT '--'; +-- +DESCRIBE (SELECT test_table_join_1.*, test_table_join_2.* FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id); +test_table_join_1.id UInt64 +test_table_join_1.value String +value_join_1 String +test_table_join_2.id UInt64 +test_table_join_2.value String +value_join_2 String +SELECT '--'; +-- +DESCRIBE (SELECT t1.*, t2.* FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id); +t1.id UInt64 +t1.value String +value_join_1 String +t2.id UInt64 +t2.value String +value_join_2 String +SELECT '--'; +-- +DESCRIBE (SELECT test_table_join_1.* APPLY toString, test_table_join_2.* APPLY toString FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id); +toString(t1.id) String +toString(t1.value) String +toString(value_join_1) String +toString(t2.id) String +toString(t2.value) String +toString(value_join_2) String +SELECT '--'; +-- +DESCRIBE (SELECT test_table_join_1.* APPLY x -> toString(x), test_table_join_2.* APPLY x -> toString(x) FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id); +toString(t1.id) String +toString(t1.value) String +toString(value_join_1) String +toString(t2.id) String +toString(t2.value) String +toString(value_join_2) String +SELECT '--'; +-- +DESCRIBE (SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_1.value_join_1, test_table_join_2.id, test_table_join_2.value, test_table_join_2.value_join_2 +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id); +test_table_join_1.id UInt64 +test_table_join_1.value String +value_join_1 String +test_table_join_2.id UInt64 +test_table_join_2.value String +value_join_2 String +SELECT '--'; +-- +DESCRIBE (SELECT t1.id, t1.value, t1.value_join_1, t2.id, t2.value, t2.value_join_2 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id); +t1.id UInt64 +t1.value String +value_join_1 String +t2.id UInt64 +t2.value String +value_join_2 String +SELECT 'Multiple JOINS'; +Multiple JOINS +DESCRIBE (SELECT * FROM test_table_join_1, test_table_join_2, test_table_join_3); +test_table_join_1.id UInt64 +test_table_join_1.value String +value_join_1 String +test_table_join_2.id UInt64 +test_table_join_2.value String +value_join_2 String +test_table_join_3.id UInt64 +test_table_join_3.value String +value_join_3 String +SELECT '--'; +-- +DESCRIBE (SELECT * FROM test_table_join_1 AS t1, test_table_join_2 AS t2, test_table_join_3 AS t3); +t1.id UInt64 +t1.value String +value_join_1 String +t2.id UInt64 +t2.value String +value_join_2 String +t3.id UInt64 +t3.value String +value_join_3 String +SELECT '--'; +-- +DESCRIBE (SELECT * APPLY toString FROM test_table_join_1 AS t1, test_table_join_2 AS t2, test_table_join_3 AS t3); +toString(t1.id) String +toString(t1.value) String +toString(value_join_1) String +toString(t2.id) String +toString(t2.value) String +toString(value_join_2) String +toString(t3.id) String +toString(t3.value) String +toString(value_join_3) String +SELECT '--'; +-- +DESCRIBE (SELECT * APPLY x -> toString(x) FROM test_table_join_1 AS t1, test_table_join_2 AS t2, test_table_join_3 AS t3); +toString(t1.id) String +toString(t1.value) String +toString(value_join_1) String +toString(t2.id) String +toString(t2.value) String +toString(value_join_2) String +toString(t3.id) String +toString(t3.value) String +toString(value_join_3) String +SELECT '--'; +-- +DESCRIBE (SELECT test_table_join_1.*, test_table_join_2.*, test_table_join_3.* +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id); +test_table_join_1.id UInt64 +test_table_join_1.value String +value_join_1 String +test_table_join_2.id UInt64 +test_table_join_2.value String +value_join_2 String +test_table_join_3.id UInt64 +test_table_join_3.value String +value_join_3 String +SELECT '--'; +-- +DESCRIBE (SELECT t1.*, t2.*, t3.* +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id); +t1.id UInt64 +t1.value String +value_join_1 String +t2.id UInt64 +t2.value String +value_join_2 String +t3.id UInt64 +t3.value String +value_join_3 String +SELECT '--'; +-- +DESCRIBE (SELECT test_table_join_1.* APPLY toString, test_table_join_2.* APPLY toString, test_table_join_3.* APPLY toString +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id); +toString(t1.id) String +toString(t1.value) String +toString(value_join_1) String +toString(t2.id) String +toString(t2.value) String +toString(value_join_2) String +toString(t3.id) String +toString(t3.value) String +toString(value_join_3) String +SELECT '--'; +-- +DESCRIBE (SELECT test_table_join_1.* APPLY x -> toString(x), test_table_join_2.* APPLY x -> toString(x), test_table_join_3.* APPLY x -> toString(x) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id); +toString(t1.id) String +toString(t1.value) String +toString(value_join_1) String +toString(t2.id) String +toString(t2.value) String +toString(value_join_2) String +toString(t3.id) String +toString(t3.value) String +toString(value_join_3) String +SELECT '--'; +-- +DESCRIBE (SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_1.value_join_1, test_table_join_2.id, test_table_join_2.value, test_table_join_2.value_join_2, +test_table_join_3.id, test_table_join_3.value, test_table_join_3.value_join_3 +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id); +test_table_join_1.id UInt64 +test_table_join_1.value String +value_join_1 String +test_table_join_2.id UInt64 +test_table_join_2.value String +value_join_2 String +test_table_join_3.id UInt64 +test_table_join_3.value String +value_join_3 String +SELECT '--'; +-- +DESCRIBE (SELECT t1.id, t1.value, t1.value_join_1, t2.id, t2.value, t2.value_join_2, t3.id, t3.value, t3.value_join_3 +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id); +t1.id UInt64 +t1.value String +value_join_1 String +t2.id UInt64 +t2.value String +value_join_2 String +t3.id UInt64 +t3.value String +value_join_3 String diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.sql b/tests/queries/0_stateless/02378_analyzer_projection_names.sql index ef01861c374..c17858cccf6 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.sql +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.sql @@ -18,6 +18,36 @@ CREATE TABLE test_table_compound INSERT INTO test_table_compound VALUES (0, tuple(0, 'Value')); +DROP TABLE IF EXISTS test_table_join_1; +CREATE TABLE test_table_join_1 +( + id UInt64, + value String, + value_join_1 String +) ENGINE=TinyLog; + +INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value', 'Join_1_Value'); + +DROP TABLE IF EXISTS test_table_join_2; +CREATE TABLE test_table_join_2 +( + id UInt64, + value String, + value_join_2 String +) ENGINE=TinyLog; + +INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value', 'Join_2_Value'); + +DROP TABLE IF EXISTS test_table_join_3; +CREATE TABLE test_table_join_3 +( + id UInt64, + value String, + value_join_3 String +) ENGINE=TinyLog; + +INSERT INTO test_table_join_3 VALUES (0, 'Join_3_Value', 'Join_3_Value'); + -- { echoOn } SELECT 'Constants'; @@ -56,7 +86,6 @@ SELECT '--'; DESCRIBE (SELECT tuple_value.* APPLY x -> x FROM test_table_compound); - SELECT '--'; DESCRIBE (SELECT tuple_value.* APPLY toString FROM test_table_compound); @@ -211,9 +240,100 @@ DESCRIBE (SELECT arrayMap(x -> (SELECT 1), [1,2,3]), arrayMap(x -> (SELECT 2) AS SELECT '--'; -SELECT (SELECT 1 AS a, 2 AS b) AS c, c.a, c.b; +DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b) AS c, c.a, c.b); + +SELECT 'Joins'; + +DESCRIBE (SELECT * FROM test_table_join_1, test_table_join_2); + +SELECT '--'; + +DESCRIBE (SELECT * FROM test_table_join_1 AS t1, test_table_join_2 AS t2); + +SELECT '--'; + +DESCRIBE (SELECT * APPLY toString FROM test_table_join_1 AS t1, test_table_join_2 AS t2); + +SELECT '--'; + +DESCRIBE (SELECT * APPLY x -> toString(x) FROM test_table_join_1 AS t1, test_table_join_2 AS t2); + +SELECT '--'; + +DESCRIBE (SELECT test_table_join_1.*, test_table_join_2.* FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id); + +SELECT '--'; + +DESCRIBE (SELECT t1.*, t2.* FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id); + +SELECT '--'; + +DESCRIBE (SELECT test_table_join_1.* APPLY toString, test_table_join_2.* APPLY toString FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id); + +SELECT '--'; + +DESCRIBE (SELECT test_table_join_1.* APPLY x -> toString(x), test_table_join_2.* APPLY x -> toString(x) FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id); + +SELECT '--'; + +DESCRIBE (SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_1.value_join_1, test_table_join_2.id, test_table_join_2.value, test_table_join_2.value_join_2 +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id); + +SELECT '--'; + +DESCRIBE (SELECT t1.id, t1.value, t1.value_join_1, t2.id, t2.value, t2.value_join_2 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id); + +SELECT 'Multiple JOINS'; + +DESCRIBE (SELECT * FROM test_table_join_1, test_table_join_2, test_table_join_3); + +SELECT '--'; + +DESCRIBE (SELECT * FROM test_table_join_1 AS t1, test_table_join_2 AS t2, test_table_join_3 AS t3); + +SELECT '--'; + +DESCRIBE (SELECT * APPLY toString FROM test_table_join_1 AS t1, test_table_join_2 AS t2, test_table_join_3 AS t3); + +SELECT '--'; + +DESCRIBE (SELECT * APPLY x -> toString(x) FROM test_table_join_1 AS t1, test_table_join_2 AS t2, test_table_join_3 AS t3); + +SELECT '--'; + +DESCRIBE (SELECT test_table_join_1.*, test_table_join_2.*, test_table_join_3.* +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id +INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id); + +SELECT '--'; + +DESCRIBE (SELECT t1.*, t2.*, t3.* +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id); + +SELECT '--'; + +DESCRIBE (SELECT test_table_join_1.* APPLY toString, test_table_join_2.* APPLY toString, test_table_join_3.* APPLY toString +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id); + +SELECT '--'; + +DESCRIBE (SELECT test_table_join_1.* APPLY x -> toString(x), test_table_join_2.* APPLY x -> toString(x), test_table_join_3.* APPLY x -> toString(x) +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id); + +SELECT '--'; + +DESCRIBE (SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_1.value_join_1, test_table_join_2.id, test_table_join_2.value, test_table_join_2.value_join_2, +test_table_join_3.id, test_table_join_3.value, test_table_join_3.value_join_3 +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id); + +SELECT '--'; + +DESCRIBE (SELECT t1.id, t1.value, t1.value_join_1, t2.id, t2.value, t2.value_join_2, t3.id, t3.value, t3.value_join_3 +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id); -- { echoOff } +DROP TABLE test_table_join_1; +DROP TABLE test_table_join_2; DROP TABLE test_table; DROP TABLE test_table_compound; From fc860ec0507012d4bcd3b20206f2255785a46d6c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 5 Sep 2022 12:24:54 +0200 Subject: [PATCH 095/188] Planner collect table expression data --- ...PlannerCollectSets.cpp => CollectSets.cpp} | 2 +- .../{PlannerCollectSets.h => CollectSets.h} | 0 src/Planner/CollectTableExpressionData.cpp | 124 +++++++++ src/Planner/CollectTableExpressionData.h | 17 ++ src/Planner/Planner.cpp | 255 ++---------------- src/Planner/PlannerAggregation.cpp | 59 ++++ src/Planner/PlannerAggregation.h | 8 + src/Planner/PlannerContext.h | 105 +------- src/Planner/TableExpressionData.h | 123 +++++++++ src/Planner/Utils.cpp | 92 +++++++ src/Planner/Utils.h | 21 +- 11 files changed, 473 insertions(+), 333 deletions(-) rename src/Planner/{PlannerCollectSets.cpp => CollectSets.cpp} (98%) rename src/Planner/{PlannerCollectSets.h => CollectSets.h} (100%) create mode 100644 src/Planner/CollectTableExpressionData.cpp create mode 100644 src/Planner/CollectTableExpressionData.h create mode 100644 src/Planner/TableExpressionData.h diff --git a/src/Planner/PlannerCollectSets.cpp b/src/Planner/CollectSets.cpp similarity index 98% rename from src/Planner/PlannerCollectSets.cpp rename to src/Planner/CollectSets.cpp index 377575f3a80..8f615b7ce35 100644 --- a/src/Planner/PlannerCollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -1,4 +1,4 @@ -#include +#include #include diff --git a/src/Planner/PlannerCollectSets.h b/src/Planner/CollectSets.h similarity index 100% rename from src/Planner/PlannerCollectSets.h rename to src/Planner/CollectSets.h diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp new file mode 100644 index 00000000000..3c073ee03bb --- /dev/null +++ b/src/Planner/CollectTableExpressionData.cpp @@ -0,0 +1,124 @@ +#include + +#include + +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace +{ + +class CollectSourceColumnsMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + PlannerContext & planner_context; + }; + + static void visit(QueryTreeNodePtr & node, Data & data) + { + auto * column_node = node->as(); + if (!column_node) + return; + + auto column_source_node = column_node->getColumnSource(); + auto column_source_node_type = column_source_node->getNodeType(); + + if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN || + column_source_node_type == QueryTreeNodeType::LAMBDA) + return; + + /// JOIN using expression + if (column_node->hasExpression() && column_source_node->getNodeType() == QueryTreeNodeType::JOIN) + return; + + auto & table_expression_node_to_data = data.planner_context.getTableExpressionNodeToData(); + + auto [it, _] = table_expression_node_to_data.emplace(column_source_node, TableExpressionData()); + auto & table_expression_columns = it->second; + + if (column_node->hasExpression()) + { + /// Replace ALIAS column with expression + table_expression_columns.addAliasColumnName(column_node->getColumnName()); + node = column_node->getExpression(); + visit(node, data); + return; + } + + if (column_source_node_type != QueryTreeNodeType::TABLE && + column_source_node_type != QueryTreeNodeType::TABLE_FUNCTION && + column_source_node_type != QueryTreeNodeType::QUERY && + column_source_node_type != QueryTreeNodeType::UNION) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected table, table function, query or union column source. Actual {}", + column_source_node->formatASTForErrorMessage()); + + bool column_already_exists = table_expression_columns.hasColumn(column_node->getColumnName()); + + if (!column_already_exists) + { + auto column_identifier = data.planner_context.getColumnUniqueIdentifier(column_source_node, column_node->getColumnName()); + data.planner_context.registerColumnNode(node, column_identifier); + table_expression_columns.addColumn(column_node->getColumn(), column_identifier); + } + else + { + auto column_identifier = table_expression_columns.getColumnIdentifierOrThrow(column_node->getColumnName()); + data.planner_context.registerColumnNode(node, column_identifier); + } + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) + { + return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; + } +}; + +using CollectSourceColumnsVisitor = CollectSourceColumnsMatcher::Visitor; + +} + +void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContext & planner_context) +{ + auto & query_node_typed = query_node->as(); + auto table_expressions_nodes = extractTableExpressions(query_node_typed.getJoinTree()); + auto & table_expression_node_to_data = planner_context.getTableExpressionNodeToData(); + + for (auto & table_expression_node : table_expressions_nodes) + { + auto [it, _] = table_expression_node_to_data.emplace(table_expression_node, TableExpressionData()); + + if (auto * table_node = table_expression_node->as()) + { + bool storage_is_remote = table_node->getStorage()->isRemote(); + it->second.setIsRemote(storage_is_remote); + } + else if (auto * table_function_node = table_expression_node->as()) + { + bool storage_is_remote = table_function_node->getStorage()->isRemote(); + it->second.setIsRemote(storage_is_remote); + } + + if (it->second.isRemote()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Remote storages are not supported"); + } + + CollectSourceColumnsVisitor::Data data {planner_context}; + CollectSourceColumnsVisitor collect_source_columns_visitor(data); + collect_source_columns_visitor.visit(query_node); +} + +} diff --git a/src/Planner/CollectTableExpressionData.h b/src/Planner/CollectTableExpressionData.h new file mode 100644 index 00000000000..f4e2d579dca --- /dev/null +++ b/src/Planner/CollectTableExpressionData.h @@ -0,0 +1,17 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +/** Collect table expression data for query node. + * Collected table expression data is registered in planner context. + * + * ALIAS table column nodes are registered in table expression data and replaced in query tree with inner alias expression. + */ +void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContext & planner_context); + +} diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 65c6e6110e5..775989cac9c 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -65,7 +65,8 @@ #include #include #include -#include +#include +#include namespace DB { @@ -106,218 +107,6 @@ namespace ErrorCodes namespace { -StreamLocalLimits getLimitsForStorage(const Settings & settings, const SelectQueryOptions & options) -{ - StreamLocalLimits limits; - limits.mode = LimitsMode::LIMITS_TOTAL; - limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read, settings.read_overflow_mode); - limits.speed_limits.max_execution_time = settings.max_execution_time; - limits.timeout_overflow_mode = settings.timeout_overflow_mode; - - /** Quota and minimal speed restrictions are checked on the initiating server of the request, and not on remote servers, - * because the initiating server has a summary of the execution of the request on all servers. - * - * But limits on data size to read and maximum execution time are reasonable to check both on initiator and - * additionally on each remote server, because these limits are checked per block of data processed, - * and remote servers may process way more blocks of data than are received by initiator. - * - * The limits to throttle maximum execution speed is also checked on all servers. - */ - if (options.to_stage == QueryProcessingStage::Complete) - { - limits.speed_limits.min_execution_rps = settings.min_execution_speed; - limits.speed_limits.min_execution_bps = settings.min_execution_speed_bytes; - } - - limits.speed_limits.max_execution_rps = settings.max_execution_speed; - limits.speed_limits.max_execution_bps = settings.max_execution_speed_bytes; - limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; - - return limits; -} - -StorageLimits getStorageLimits(const Context & context, const SelectQueryOptions & options) -{ - const auto & settings = context.getSettingsRef(); - - StreamLocalLimits limits; - SizeLimits leaf_limits; - - /// Set the limits and quota for reading data, the speed and time of the query. - if (!options.ignore_limits) - { - limits = getLimitsForStorage(settings, options); - leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, settings.max_bytes_to_read_leaf, settings.read_overflow_mode_leaf); - } - - return {limits, leaf_limits}; -} - -/** There are no limits on the maximum size of the result for the subquery. - * Since the result of the query is not the result of the entire query. - */ -ContextPtr buildSubqueryContext(const ContextPtr & context) -{ - /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. - * Because the result of this query is not the result of the entire query. - * Constraints work instead - * max_rows_in_set, max_bytes_in_set, set_overflow_mode, - * max_rows_in_join, max_bytes_in_join, join_overflow_mode, - * which are checked separately (in the Set, Join objects). - */ - auto subquery_context = Context::createCopy(context); - Settings subquery_settings = context->getSettings(); - subquery_settings.max_result_rows = 0; - subquery_settings.max_result_bytes = 0; - /// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query). - subquery_settings.extremes = false; - subquery_context->setSettings(subquery_settings); - return subquery_context; -} - -QueryTreeNodes extractAggregateFunctionNodes(const QueryTreeNodePtr & query_node) -{ - const auto & query_node_typed = query_node->as(); - - QueryTreeNodes aggregate_function_nodes; - if (query_node_typed.hasHaving()) - collectAggregateFunctionNodes(query_node_typed.getHaving(), aggregate_function_nodes); - - if (query_node_typed.hasOrderBy()) - collectAggregateFunctionNodes(query_node_typed.getOrderByNode(), aggregate_function_nodes); - - collectAggregateFunctionNodes(query_node_typed.getProjectionNode(), aggregate_function_nodes); - - return aggregate_function_nodes; -} - -AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodes & aggregate_function_nodes, const PlannerContext & planner_context) -{ - QueryTreeNodeToName node_to_name; - NameSet unique_aggregate_action_node_names; - AggregateDescriptions aggregate_descriptions; - - for (const auto & aggregate_function_node : aggregate_function_nodes) - { - const auto & aggregagte_function_node_typed = aggregate_function_node->as(); - String node_name = calculateActionNodeName(aggregate_function_node, planner_context, node_to_name); - auto [_, inserted] = unique_aggregate_action_node_names.emplace(node_name); - if (!inserted) - continue; - - AggregateDescription aggregate_description; - aggregate_description.function = aggregagte_function_node_typed.getAggregateFunction(); - - const auto & parameters_nodes = aggregagte_function_node_typed.getParameters().getNodes(); - aggregate_description.parameters.reserve(parameters_nodes.size()); - - for (const auto & parameter_node : parameters_nodes) - { - /// Function parameters constness validated during analysis stage - aggregate_description.parameters.push_back(parameter_node->getConstantValue().getValue()); - } - - const auto & arguments_nodes = aggregagte_function_node_typed.getArguments().getNodes(); - aggregate_description.argument_names.reserve(arguments_nodes.size()); - - for (const auto & argument_node : arguments_nodes) - { - String argument_node_name = calculateActionNodeName(argument_node, planner_context, node_to_name); - aggregate_description.argument_names.emplace_back(std::move(argument_node_name)); - } - - aggregate_description.column_name = node_name; - aggregate_descriptions.push_back(std::move(aggregate_description)); - } - - return aggregate_descriptions; -} - -class CollectSourceColumnsMatcher -{ -public: - using Visitor = InDepthQueryTreeVisitor; - - struct Data - { - PlannerContext & planner_context; - }; - - static void visit(QueryTreeNodePtr & node, Data & data) - { - auto * column_node = node->as(); - if (!column_node) - return; - - auto column_source_node = column_node->getColumnSource(); - auto column_source_node_type = column_source_node->getNodeType(); - - if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN || - column_source_node_type == QueryTreeNodeType::LAMBDA) - return; - - /// JOIN using expression - if (column_node->hasExpression() && column_source_node->getNodeType() == QueryTreeNodeType::JOIN) - return; - - auto & table_expression_node_to_columns = data.planner_context.getTableExpressionNodeToColumns(); - - auto [it, _] = table_expression_node_to_columns.emplace(column_source_node, TableExpressionColumns()); - auto & table_expression_columns = it->second; - - if (column_node->hasExpression()) - { - /// Replace ALIAS column with expression - table_expression_columns.addAliasColumnName(column_node->getColumnName()); - node = column_node->getExpression(); - visit(node, data); - return; - } - - if (column_source_node_type != QueryTreeNodeType::TABLE && - column_source_node_type != QueryTreeNodeType::TABLE_FUNCTION && - column_source_node_type != QueryTreeNodeType::QUERY && - column_source_node_type != QueryTreeNodeType::UNION) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected table, table function, query or union column source. Actual {}", - column_source_node->formatASTForErrorMessage()); - - bool column_already_exists = table_expression_columns.hasColumn(column_node->getColumnName()); - - if (!column_already_exists) - { - auto column_identifier = data.planner_context.getColumnUniqueIdentifier(column_source_node, column_node->getColumnName()); - data.planner_context.registerColumnNode(node, column_identifier); - table_expression_columns.addColumn(column_node->getColumn(), column_identifier); - } - else - { - auto column_identifier = table_expression_columns.getColumnIdentifierOrThrow(column_node->getColumnName()); - data.planner_context.registerColumnNode(node, column_identifier); - } - } - - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) - { - return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; - } -}; - -using CollectSourceColumnsVisitor = CollectSourceColumnsMatcher::Visitor; - -ActionsDAGPtr convertExpressionNodeIntoDAG(const QueryTreeNodePtr & expression_node, const ColumnsWithTypeAndName & inputs, const PlannerContextPtr & planner_context) -{ - ActionsDAGPtr action_dag = std::make_shared(inputs); - PlannerActionsVisitor actions_visitor(planner_context); - auto expression_dag_index_nodes = actions_visitor.visit(action_dag, expression_node); - action_dag->getOutputs().clear(); - - for (auto & expression_dag_index_node : expression_dag_index_nodes) - action_dag->getOutputs().push_back(expression_dag_index_node); - - return action_dag; -} - QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, @@ -335,13 +124,14 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, QueryPlan query_plan; - auto & table_expression_node_to_columns = planner_context->getTableExpressionNodeToColumns(); + auto & table_expression_node_to_data = planner_context->getTableExpressionNodeToData(); + auto it = table_expression_node_to_data.find(table_expression); + if (it == table_expression_node_to_data.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Table expression {} is not registered", + table_expression->formatASTForErrorMessage()); - /** Use default columns to support case when there are no columns in query. - * Example: SELECT 1; - */ - const auto & [it, _] = table_expression_node_to_columns.emplace(table_expression, TableExpressionColumns()); - auto & table_expression_columns = it->second; + auto & table_expression_data = it->second; if (table_node || table_function_node) { @@ -355,7 +145,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, table_expression_query_info.table_expression_modifiers = table_function_node->getTableExpressionModifiers(); auto from_stage = storage->getQueryProcessingStage(planner_context->getQueryContext(), select_query_options.to_stage, storage_snapshot, table_expression_query_info); - const auto & columns_names = table_expression_columns.getColumnsNames(); + const auto & columns_names = table_expression_data.getColumnsNames(); Names column_names(columns_names.begin(), columns_names.end()); std::optional read_additional_column; @@ -370,7 +160,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, { auto column_identifier = planner_context->getColumnUniqueIdentifier(table_expression, read_additional_column->name); column_names.push_back(read_additional_column->name); - table_expression_columns.addColumn(*read_additional_column, column_identifier); + table_expression_data.addColumn(*read_additional_column, column_identifier); } if (!column_names.empty()) @@ -406,7 +196,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, auto rename_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); - for (const auto & [column_name, column_identifier] : table_expression_columns.getColumnNameToIdentifier()) + for (const auto & [column_name, column_identifier] : table_expression_data.getColumnNameToIdentifier()) { auto position = query_plan.getCurrentDataStream().header.getPositionByName(column_name); const auto * node_to_rename = rename_actions_dag->getOutputs()[position]; @@ -930,13 +720,10 @@ void Planner::buildQueryPlanIfNeeded() select_query_info.query = select_query_info.original_query; StorageLimitsList storage_limits; - storage_limits.push_back(getStorageLimits(*query_context, select_query_options)); + storage_limits.push_back(buildStorageLimits(*query_context, select_query_options)); select_query_info.storage_limits = std::make_shared(storage_limits); - CollectSourceColumnsVisitor::Data data {*planner_context}; - CollectSourceColumnsVisitor collect_source_columns_visitor(data); - collect_source_columns_visitor.visit(query_tree); - + collectTableExpressionData(query_tree, *planner_context); collectSets(query_tree, *planner_context); query_plan = buildQueryPlanForJoinTreeNode(query_node.getJoinTree(), select_query_info, select_query_options, planner_context); @@ -950,7 +737,7 @@ void Planner::buildQueryPlanIfNeeded() const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & where_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - auto where_actions = convertExpressionNodeIntoDAG(query_node.getWhere(), where_input, planner_context); + auto where_actions = buildActionsDAGFromExpressionNode(query_node.getWhere(), where_input, planner_context); where_filter_action_node_name = where_actions->getOutputs().at(0)->result_name; actions_chain.addStep(std::make_unique(std::move(where_actions))); where_action_step_index = actions_chain.getLastStepIndex(); @@ -1094,7 +881,7 @@ void Planner::buildQueryPlanIfNeeded() chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & having_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - auto having_actions = convertExpressionNodeIntoDAG(query_node.getHaving(), having_input, planner_context); + auto having_actions = buildActionsDAGFromExpressionNode(query_node.getHaving(), having_input, planner_context); having_filter_action_node_name = having_actions->getOutputs().at(0)->result_name; actions_chain.addStep(std::make_unique(std::move(having_actions))); having_action_step_index = actions_chain.getLastStepIndex(); @@ -1142,7 +929,7 @@ void Planner::buildQueryPlanIfNeeded() { chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & limit_by_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - auto limit_by_actions = convertExpressionNodeIntoDAG(query_node.getLimitByNode(), limit_by_input, planner_context); + auto limit_by_actions = buildActionsDAGFromExpressionNode(query_node.getLimitByNode(), limit_by_input, planner_context); limit_by_columns_names.reserve(limit_by_actions->getOutputs().size()); for (auto & output_node : limit_by_actions->getOutputs()) @@ -1155,7 +942,7 @@ void Planner::buildQueryPlanIfNeeded() chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - auto projection_actions = convertExpressionNodeIntoDAG(query_node.getProjectionNode(), projection_input, planner_context); + auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), projection_input, planner_context); const auto & projection_action_dag_nodes = projection_actions->getOutputs(); size_t projection_action_dag_nodes_size = projection_action_dag_nodes.size(); @@ -1266,10 +1053,10 @@ void Planner::buildQueryPlanIfNeeded() : static_cast(settings.max_threads); bool storage_has_evenly_distributed_read = false; - auto & table_expression_node_to_columns = planner_context->getTableExpressionNodeToColumns(); - if (table_expression_node_to_columns.size() == 1) + auto & table_expression_node_to_data = planner_context->getTableExpressionNodeToData(); + if (table_expression_node_to_data.size() == 1) { - auto it = table_expression_node_to_columns.begin(); + auto it = table_expression_node_to_data.begin(); const auto & table_expression_node = it->first; if (const auto * table_node = table_expression_node->as()) storage_has_evenly_distributed_read = table_node->getStorage()->hasEvenlyDistributedRead(); diff --git a/src/Planner/PlannerAggregation.cpp b/src/Planner/PlannerAggregation.cpp index 14b77faec8f..8e824aa8588 100644 --- a/src/Planner/PlannerAggregation.cpp +++ b/src/Planner/PlannerAggregation.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -192,4 +193,62 @@ void resolveGroupingFunctions(QueryTreeNodePtr & query_node, planner_context); } +QueryTreeNodes extractAggregateFunctionNodes(const QueryTreeNodePtr & query_node) +{ + const auto & query_node_typed = query_node->as(); + + QueryTreeNodes aggregate_function_nodes; + if (query_node_typed.hasHaving()) + collectAggregateFunctionNodes(query_node_typed.getHaving(), aggregate_function_nodes); + + if (query_node_typed.hasOrderBy()) + collectAggregateFunctionNodes(query_node_typed.getOrderByNode(), aggregate_function_nodes); + + collectAggregateFunctionNodes(query_node_typed.getProjectionNode(), aggregate_function_nodes); + + return aggregate_function_nodes; +} + +AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodes & aggregate_function_nodes, const PlannerContext & planner_context) +{ + QueryTreeNodeToName node_to_name; + NameSet unique_aggregate_action_node_names; + AggregateDescriptions aggregate_descriptions; + + for (const auto & aggregate_function_node : aggregate_function_nodes) + { + const auto & aggregagte_function_node_typed = aggregate_function_node->as(); + String node_name = calculateActionNodeName(aggregate_function_node, planner_context, node_to_name); + auto [_, inserted] = unique_aggregate_action_node_names.emplace(node_name); + if (!inserted) + continue; + + AggregateDescription aggregate_description; + aggregate_description.function = aggregagte_function_node_typed.getAggregateFunction(); + + const auto & parameters_nodes = aggregagte_function_node_typed.getParameters().getNodes(); + aggregate_description.parameters.reserve(parameters_nodes.size()); + + for (const auto & parameter_node : parameters_nodes) + { + /// Function parameters constness validated during analysis stage + aggregate_description.parameters.push_back(parameter_node->getConstantValue().getValue()); + } + + const auto & arguments_nodes = aggregagte_function_node_typed.getArguments().getNodes(); + aggregate_description.argument_names.reserve(arguments_nodes.size()); + + for (const auto & argument_node : arguments_nodes) + { + String argument_node_name = calculateActionNodeName(argument_node, planner_context, node_to_name); + aggregate_description.argument_names.emplace_back(std::move(argument_node_name)); + } + + aggregate_description.column_name = node_name; + aggregate_descriptions.push_back(std::move(aggregate_description)); + } + + return aggregate_descriptions; +} + } diff --git a/src/Planner/PlannerAggregation.h b/src/Planner/PlannerAggregation.h index 45858f03450..b4fc449400b 100644 --- a/src/Planner/PlannerAggregation.h +++ b/src/Planner/PlannerAggregation.h @@ -1,8 +1,10 @@ #pragma once #include +#include #include +#include #include @@ -18,4 +20,10 @@ void resolveGroupingFunctions(QueryTreeNodePtr & query_node, const GroupingSetsParamsList & grouping_sets_parameters_list, const PlannerContext & planner_context); +/// Extract aggregate function nodes from query node +QueryTreeNodes extractAggregateFunctionNodes(const QueryTreeNodePtr & query_node); + +/// Extract aggregate descriptions from aggregate function nodes +AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodes & aggregate_function_nodes, const PlannerContext & planner_context); + } diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 95af4858eb2..d8357046646 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -11,6 +11,8 @@ #include +#include + namespace DB { @@ -19,97 +21,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -using ColumnIdentifier = std::string; - -class TableExpressionColumns -{ -public: - using ColumnNameToColumnIdentifier = std::unordered_map; - - bool hasColumn(const std::string & column_name) const - { - return alias_columns_names.contains(column_name) || columns_names.contains(column_name); - } - - void addColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) - { - if (hasColumn(column.name)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Column with name {} already exists"); - - columns_names.insert(column.name); - columns.push_back(column); - column_name_to_column_identifier.emplace(column.name, column_identifier); - } - - void addColumnIfNotExists(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) - { - if (hasColumn(column.name)) - return; - - columns_names.insert(column.name); - columns.push_back(column); - column_name_to_column_identifier.emplace(column.name, column_identifier); - } - - void addAliasColumnName(const std::string & column_name) - { - alias_columns_names.insert(column_name); - } - - const NameSet & getAliasColumnsNames() const - { - return alias_columns_names; - } - - const NameSet & getColumnsNames() const - { - return columns_names; - } - - const NamesAndTypesList & getColumns() const - { - return columns; - } - - const ColumnNameToColumnIdentifier & getColumnNameToIdentifier() const - { - return column_name_to_column_identifier; - } - - const ColumnIdentifier & getColumnIdentifierOrThrow(const std::string & column_name) const - { - auto it = column_name_to_column_identifier.find(column_name); - if (it == column_name_to_column_identifier.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Column identifier for name {} does not exists", - column_name); - - return it->second; - } - - const ColumnIdentifier * getColumnIdentifierOrNull(const std::string & column_name) const - { - auto it = column_name_to_column_identifier.find(column_name); - if (it == column_name_to_column_identifier.end()) - return nullptr; - - return &it->second; - } - -private: - /// Valid for table, table function, query table expression nodes - NamesAndTypesList columns; - - /// Valid for table, table function, query table expression nodes - NameSet columns_names; - - /// Valid only for table table expression node - NameSet alias_columns_names; - - /// Valid for table, table function, query table expression nodes - ColumnNameToColumnIdentifier column_name_to_column_identifier; -}; - /// Subquery node for set struct SubqueryNodeForSet { @@ -181,14 +92,14 @@ public: return global_planner_context; } - const std::unordered_map & getTableExpressionNodeToColumns() const + const std::unordered_map & getTableExpressionNodeToData() const { - return table_expression_node_to_columns; + return table_expression_node_to_data; } - std::unordered_map & getTableExpressionNodeToColumns() + std::unordered_map & getTableExpressionNodeToData() { - return table_expression_node_to_columns; + return table_expression_node_to_data; } ColumnIdentifier getColumnUniqueIdentifier(const QueryTreeNodePtr & column_source_node, std::string column_name = {}); @@ -209,8 +120,8 @@ private: /// Column node to column identifier std::unordered_map column_node_to_column_identifier; - /// Table expression node to columns - std::unordered_map table_expression_node_to_columns; + /// Table expression node to data + std::unordered_map table_expression_node_to_data; size_t column_identifier_counter = 0; }; diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h new file mode 100644 index 00000000000..2708279e1dd --- /dev/null +++ b/src/Planner/TableExpressionData.h @@ -0,0 +1,123 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +using ColumnIdentifier = std::string; + +class TableExpressionData +{ +public: + using ColumnNameToColumnIdentifier = std::unordered_map; + + bool hasColumn(const std::string & column_name) const + { + return alias_columns_names.contains(column_name) || columns_names.contains(column_name); + } + + void addColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) + { + if (hasColumn(column.name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Column with name {} already exists"); + + columns_names.insert(column.name); + columns.push_back(column); + column_name_to_column_identifier.emplace(column.name, column_identifier); + } + + void addColumnIfNotExists(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) + { + if (hasColumn(column.name)) + return; + + columns_names.insert(column.name); + columns.push_back(column); + column_name_to_column_identifier.emplace(column.name, column_identifier); + } + + void addAliasColumnName(const std::string & column_name) + { + alias_columns_names.insert(column_name); + } + + const NameSet & getAliasColumnsNames() const + { + return alias_columns_names; + } + + const NameSet & getColumnsNames() const + { + return columns_names; + } + + const NamesAndTypesList & getColumns() const + { + return columns; + } + + const ColumnNameToColumnIdentifier & getColumnNameToIdentifier() const + { + return column_name_to_column_identifier; + } + + const ColumnIdentifier & getColumnIdentifierOrThrow(const std::string & column_name) const + { + auto it = column_name_to_column_identifier.find(column_name); + if (it == column_name_to_column_identifier.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column identifier for name {} does not exists", + column_name); + + return it->second; + } + + const ColumnIdentifier * getColumnIdentifierOrNull(const std::string & column_name) const + { + auto it = column_name_to_column_identifier.find(column_name); + if (it == column_name_to_column_identifier.end()) + return nullptr; + + return &it->second; + } + + /** Cache value of storage is remote method call. + * + * Valid only for table and table function node. + */ + bool isRemote() const + { + return is_remote; + } + + /// Set is remote value + void setIsRemote(bool is_remote_value) + { + is_remote = is_remote_value; + } + +private: + /// Valid for table, table function, query table expression nodes + NamesAndTypesList columns; + + /// Valid for table, table function, query table expression nodes + NameSet columns_names; + + /// Valid only for table table expression node + NameSet alias_columns_names; + + /// Valid for table, table function, query table expression nodes + ColumnNameToColumnIdentifier column_name_to_column_identifier; + + /// Cached value if table expression receives data from remote server + bool is_remote = false; +}; + +} diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 96a631612e7..6ce43097d1c 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -8,10 +8,14 @@ #include +#include + #include #include #include +#include + namespace DB { @@ -87,4 +91,92 @@ ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node) return result_ast; } +/** There are no limits on the maximum size of the result for the subquery. + * Since the result of the query is not the result of the entire query. + */ +ContextPtr buildSubqueryContext(const ContextPtr & context) +{ + /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. + * Because the result of this query is not the result of the entire query. + * Constraints work instead + * max_rows_in_set, max_bytes_in_set, set_overflow_mode, + * max_rows_in_join, max_bytes_in_join, join_overflow_mode, + * which are checked separately (in the Set, Join objects). + */ + auto subquery_context = Context::createCopy(context); + Settings subquery_settings = context->getSettings(); + subquery_settings.max_result_rows = 0; + subquery_settings.max_result_bytes = 0; + /// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query). + subquery_settings.extremes = false; + subquery_context->setSettings(subquery_settings); + + return subquery_context; +} + +namespace +{ + +StreamLocalLimits getLimitsForStorage(const Settings & settings, const SelectQueryOptions & options) +{ + StreamLocalLimits limits; + limits.mode = LimitsMode::LIMITS_TOTAL; + limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read, settings.read_overflow_mode); + limits.speed_limits.max_execution_time = settings.max_execution_time; + limits.timeout_overflow_mode = settings.timeout_overflow_mode; + + /** Quota and minimal speed restrictions are checked on the initiating server of the request, and not on remote servers, + * because the initiating server has a summary of the execution of the request on all servers. + * + * But limits on data size to read and maximum execution time are reasonable to check both on initiator and + * additionally on each remote server, because these limits are checked per block of data processed, + * and remote servers may process way more blocks of data than are received by initiator. + * + * The limits to throttle maximum execution speed is also checked on all servers. + */ + if (options.to_stage == QueryProcessingStage::Complete) + { + limits.speed_limits.min_execution_rps = settings.min_execution_speed; + limits.speed_limits.min_execution_bps = settings.min_execution_speed_bytes; + } + + limits.speed_limits.max_execution_rps = settings.max_execution_speed; + limits.speed_limits.max_execution_bps = settings.max_execution_speed_bytes; + limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; + + return limits; +} + +} + +StorageLimits buildStorageLimits(const Context & context, const SelectQueryOptions & options) +{ + const auto & settings = context.getSettingsRef(); + + StreamLocalLimits limits; + SizeLimits leaf_limits; + + /// Set the limits and quota for reading data, the speed and time of the query. + if (!options.ignore_limits) + { + limits = getLimitsForStorage(settings, options); + leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, settings.max_bytes_to_read_leaf, settings.read_overflow_mode_leaf); + } + + return {limits, leaf_limits}; +} + +ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & expression_node, const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context) +{ + ActionsDAGPtr action_dag = std::make_shared(input_columns); + PlannerActionsVisitor actions_visitor(planner_context); + auto expression_dag_index_nodes = actions_visitor.visit(action_dag, expression_node); + action_dag->getOutputs().clear(); + + for (auto & expression_dag_index_node : expression_dag_index_nodes) + action_dag->getOutputs().push_back(expression_dag_index_node); + + return action_dag; +} + } diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 4902aa45e4a..19ec8c4a76a 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -4,10 +4,15 @@ #include -#include +#include +#include #include +#include + +#include + #include namespace DB @@ -25,4 +30,18 @@ Block buildCommonHeaderForUnion(const Blocks & queries_headers); /// Convert query node to ASTSelectQuery ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node); +/// Build context for subquery execution +ContextPtr buildSubqueryContext(const ContextPtr & context); + +/// Build limits for storage +StorageLimits buildStorageLimits(const Context & context, const SelectQueryOptions & options); + +/** Convert query tree expression node into actions dag. + * Inputs are not used for actions dag outputs. + * Only root query tree expression node is used as actions dag output. + */ +ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & expression_node, + const ColumnsWithTypeAndName & input_columns, + const PlannerContextPtr & planner_context); + } From ca93ee747930d670c3707a86bbb05e4055d3a5b3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 6 Sep 2022 17:25:52 +0200 Subject: [PATCH 096/188] Fixed tests --- src/Analyzer/ArrayJoinNode.cpp | 5 - .../CollectAggregateFunctionVisitor.cpp | 2 +- src/Analyzer/ColumnNode.h | 5 + src/Analyzer/ConstantNode.cpp | 6 +- src/Analyzer/ConstantNode.h | 5 +- src/Analyzer/IQueryTreeNode.h | 1 + src/Analyzer/JoinNode.cpp | 5 - src/Analyzer/QueryAnalysisPass.cpp | 45 +++++--- src/Analyzer/QueryNode.h | 5 + src/Analyzer/QueryTreeBuilder.cpp | 3 - src/Analyzer/SetUtils.cpp | 1 + src/Analyzer/TableFunctionNode.cpp | 10 +- src/Analyzer/TableFunctionNode.h | 7 +- src/Analyzer/UnionNode.h | 5 + src/Analyzer/Utils.cpp | 5 + .../InterpreterSelectQueryAnalyzer.cpp | 4 - src/Interpreters/TableJoin.cpp | 86 ++++++-------- src/Interpreters/TableJoin.h | 17 ++- .../getHeaderForProcessingStage.cpp | 5 + src/Planner/CollectSets.cpp | 11 +- src/Planner/CollectTableExpressionData.cpp | 8 +- src/Planner/Planner.cpp | 79 +++++++++++-- src/Planner/PlannerActionsVisitor.cpp | 106 +++++++++++------- src/Planner/PlannerAggregation.cpp | 2 +- src/Planner/PlannerContext.h | 5 - src/Planner/PlannerJoins.cpp | 32 +++++- src/Planner/PlannerJoins.h | 4 +- src/Planner/Utils.cpp | 1 + ...reuseStorageOrderingForWindowFunctions.cpp | 4 + src/Storages/MergeTree/KeyCondition.cpp | 15 ++- src/Storages/MergeTree/MergeTreeData.cpp | 5 + src/Storages/StorageDistributed.cpp | 4 + src/Storages/StorageMerge.cpp | 14 ++- .../transformQueryForExternalDatabase.cpp | 6 + ...76_analyzer_in_function_subquery.reference | 6 - .../02376_analyzer_in_function_subquery.sql | 16 +-- 36 files changed, 346 insertions(+), 194 deletions(-) diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp index 73323c7d369..d88164230e5 100644 --- a/src/Analyzer/ArrayJoinNode.cpp +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -11,11 +11,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int UNSUPPORTED_METHOD; -} - ArrayJoinNode::ArrayJoinNode(QueryTreeNodePtr table_expression_, QueryTreeNodePtr join_expressions_, bool is_left_) : is_left(is_left_) { diff --git a/src/Analyzer/CollectAggregateFunctionVisitor.cpp b/src/Analyzer/CollectAggregateFunctionVisitor.cpp index cdf6c986601..40c2f24cbc0 100644 --- a/src/Analyzer/CollectAggregateFunctionVisitor.cpp +++ b/src/Analyzer/CollectAggregateFunctionVisitor.cpp @@ -48,7 +48,7 @@ public: static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { - return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; + return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); } }; diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index 1ef680842d0..f22feb30d6e 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -7,6 +7,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + /** Column node represents column in query tree. * Column must have some column source. * Column can be table expression, lambda, subquery. diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index c187bf71ae5..380c4aba2a1 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -11,6 +11,8 @@ #include +#include + namespace DB { @@ -21,11 +23,11 @@ ConstantNode::ConstantNode(ConstantValuePtr constant_value_) } ConstantNode::ConstantNode(Field value_, DataTypePtr value_data_type_) - : ConstantNode(std::make_shared(std::move(value_), std::move(value_data_type_))) + : ConstantNode(std::make_shared(convertFieldToTypeOrThrow(value_, *value_data_type_), value_data_type_)) {} ConstantNode::ConstantNode(Field value_) - : ConstantNode(std::make_shared(value_, applyVisitor(FieldToDataType(), value_))) + : ConstantNode(value_, applyVisitor(FieldToDataType(), value_)) {} void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index 41a58a4caf9..4ebc53d86af 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -20,7 +20,10 @@ public: /// Construct constant query tree node from constant value explicit ConstantNode(ConstantValuePtr constant_value_); - /// Construct constant query tree node from field and data type + /** Construct constant query tree node from field and data type. + * + * Throws exception if value cannot be converted to value data type. + */ explicit ConstantNode(Field value_, DataTypePtr value_data_type_); /// Construct constant query tree node from field, data type will be derived from field value diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index b720a4ff90c..0bfd73085df 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -21,6 +21,7 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; + extern const int LOGICAL_ERROR; } class WriteBuffer; diff --git a/src/Analyzer/JoinNode.cpp b/src/Analyzer/JoinNode.cpp index e723bbc2d9a..fcf2bf4c8d5 100644 --- a/src/Analyzer/JoinNode.cpp +++ b/src/Analyzer/JoinNode.cpp @@ -15,11 +15,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int UNSUPPORTED_METHOD; -} - JoinNode::JoinNode(QueryTreeNodePtr left_table_expression_, QueryTreeNodePtr right_table_expression_, QueryTreeNodePtr join_expression_, diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index e8da63e0614..b4eebf289a7 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -90,7 +90,6 @@ namespace ErrorCodes extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS; extern const int UNKNOWN_DATABASE; extern const int UNKNOWN_TABLE; - extern const int INCORRECT_ELEMENT_OF_SET; extern const int TYPE_MISMATCH; extern const int AMBIGUOUS_IDENTIFIER; extern const int INVALID_WITH_FILL_EXPRESSION; @@ -103,6 +102,8 @@ namespace ErrorCodes extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_FINAL; extern const int SAMPLING_NOT_SUPPORTED; + extern const int NO_COMMON_TYPE; + extern const int NOT_IMPLEMENTED; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -193,6 +194,7 @@ namespace ErrorCodes * TODO: JOIN support SELF JOIN with MergeTree. JOIN support matchers. * TODO: WINDOW functions * TODO: Support function identifier resolve from parent query scope, if lambda in parent scope does not capture any columns. + * TODO: Support group_by_use_nulls */ namespace @@ -2736,7 +2738,11 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc /// Replace right IN function argument if it is table or table function with subquery that read ordinary columns if (is_special_function_in) { - auto & in_second_argument = function_node.getArguments().getNodes().at(1); + auto & function_in_arguments_nodes = function_node.getArguments().getNodes(); + if (function_in_arguments_nodes.size() != 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} expects 2 arguments", function_name); + + auto & in_second_argument = function_in_arguments_nodes[1]; auto * table_node = in_second_argument->as(); auto * table_function_node = in_second_argument->as(); auto * query_node = in_second_argument->as(); @@ -3381,16 +3387,17 @@ void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, Iden for (auto & node : node_list_typed.getNodes()) { - resolveExpressionNode(node, scope, allow_lambda_expression, allow_table_expression); + auto node_to_resolve = node; + resolveExpressionNode(node_to_resolve, scope, allow_lambda_expression, allow_table_expression); - if (auto * expression_list = node->as()) + if (auto * expression_list = node_to_resolve->as()) { for (auto & expression_list_node : expression_list->getNodes()) result_nodes.push_back(std::move(expression_list_node)); } else { - result_nodes.push_back(std::move(node)); + result_nodes.push_back(std::move(node_to_resolve)); } } @@ -3737,16 +3744,18 @@ NamesAndTypes QueryAnalyzer::resolveProjectionExpressionNodeList(QueryTreeNodePt auto & node = initial_node_list_nodes_copy[i]; String display_name = calculateProjectionNodeDisplayName(node, scope); - resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - if (auto * expression_list = node->as()) + auto node_to_resolve = node; + resolveExpressionNode(node_to_resolve, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + if (auto * expression_list = node_to_resolve->as()) { for (auto & expression_list_node : expression_list->getNodes()) projection_nodes_with_display_name.emplace_back(expression_list_node, calculateProjectionNodeDisplayName(expression_list_node, scope)); } else { - projection_nodes_with_display_name.emplace_back(node, display_name); + projection_nodes_with_display_name.emplace_back(node_to_resolve, std::move(display_name)); } } @@ -4222,19 +4231,14 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, identifier_full_name, scope.scope_node->formatASTForErrorMessage()); - DataTypePtr common_type; + DataTypePtr common_type = tryGetLeastSupertype(DataTypes{result_left_table_expression->getResultType(), result_right_table_expression->getResultType()}); - try - { - common_type = getLeastSupertype(DataTypes{result_left_table_expression->getResultType(), result_right_table_expression->getResultType()}); - } - catch (Exception & ex) - { - ex.addMessage("JOIN {} cannot infer common type in USING for identifier {}. In scope {}", + if (!common_type) + throw Exception(ErrorCodes::NO_COMMON_TYPE, + "JOIN {} cannot infer common type in USING for identifier {}. In scope {}", join_node.formatASTForErrorMessage(), identifier_full_name, scope.scope_node->formatASTForErrorMessage()); - } NameAndTypePair join_using_columns_common_name_and_type(identifier_full_name, common_type); ListNodePtr join_using_expression = std::make_shared(QueryTreeNodes{result_left_table_expression, result_right_table_expression}); @@ -4256,7 +4260,9 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, else if (join_node.getJoinExpression()) { expressions_visitor.visit(join_node.getJoinExpression()); - resolveExpressionNode(join_node.getJoinExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + auto join_expression = join_node.getJoinExpression(); + resolveExpressionNode(join_expression, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + join_node.getJoinExpression() = std::move(join_expression); } break; @@ -4768,6 +4774,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier validate_group_by_visitor.visit(query_node_typed.getProjectionNode()); } + if (context->getSettingsRef().group_by_use_nulls) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "GROUP BY use nulls is not supported"); + bool is_rollup_or_cube = query_node_typed.isGroupByWithRollup() || query_node_typed.isGroupByWithCube(); if (!has_aggregation && (query_node_typed.isGroupByWithGroupingSets() || is_rollup_or_cube)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS, ROLLUP, CUBE or GROUPING SETS are not supported without aggregation"); diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 76769779f3b..6d7c4c2792e 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -11,6 +11,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + /** Query node represents query in query tree. */ class QueryNode; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index a845f453dbf..1e46a6083b8 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -55,10 +55,7 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; - extern const int UNKNOWN_DATABASE; - extern const int UNKNOWN_TABLE; extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; } namespace diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 0da311b800e..40d088b4c05 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -15,6 +15,7 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_ELEMENT_OF_SET; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index 0e6c1f93502..0d18d5a3d00 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -28,9 +28,6 @@ TableFunctionNode::TableFunctionNode(String table_function_name_) void TableFunctionNode::resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context) { - if (isResolved()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function node {} already resolved", table_function_name); - table_function = std::move(table_function_value); storage = std::move(storage_value); storage_id = storage->getStorageID(); @@ -130,11 +127,8 @@ ASTPtr TableFunctionNode::toASTImpl() const table_function_ast->name = table_function_name; const auto & arguments = getArguments(); - if (!arguments.getNodes().empty()) - { - table_function_ast->children.push_back(arguments.toAST()); - table_function_ast->arguments = table_function_ast->children.back(); - } + table_function_ast->children.push_back(arguments.toAST()); + table_function_ast->arguments = table_function_ast->children.back(); return table_function_ast; } diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index 0b02842f37c..6f2659bd8de 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -14,6 +14,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + class ITableFunction; using TableFunctionPtr = std::shared_ptr; @@ -84,7 +89,7 @@ public: const StoragePtr & getStorageOrThrow() const { if (!storage) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Function node is not resolved"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function node is not resolved"); return storage; } diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h index 1ca90b91bd1..7ddc0b5381c 100644 --- a/src/Analyzer/UnionNode.h +++ b/src/Analyzer/UnionNode.h @@ -13,6 +13,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + /** Union node represents union of queries in query tree. */ class UnionNode; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 37e391d0b11..948d7190499 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -17,6 +17,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + bool isNodePartOfTree(const IQueryTreeNode * node, const IQueryTreeNode * root) { std::vector nodes_to_process; diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 73dde2a4aa8..bafc2cc2644 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -63,10 +63,6 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; - extern const int LOGICAL_ERROR; - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; - extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; - extern const int INVALID_JOIN_ON_EXPRESSION; } namespace diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 7461c53a342..662af320ff9 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -46,49 +46,51 @@ namespace std::string formatTypeMap(const TableJoin::NameToTypeMap & target, const TableJoin::NameToTypeMap & source) { std::vector text; - for (const auto & [key, value] : target) + for (const auto & [k, v] : target) { - auto src_type_it = source.find(key); + auto src_type_it = source.find(k); std::string src_type_name = src_type_it != source.end() ? src_type_it->second->getName() : ""; - text.push_back(fmt::format("{} : {} -> {}", key, src_type_name, value->getName())); + text.push_back(fmt::format("{} : {} -> {}", k, src_type_name, v->getName())); } - return fmt::format("{}", fmt::join(text, ", ")); } +} + +namespace +{ + struct BothSidesTag {}; struct LeftSideTag {}; struct RightSideTag {}; -template -bool forKeyNamesInJoinClauses(JoinClauses & join_clauses, Func callback) +template +bool forAllKeys(OnExpr & expressions, Func callback) { static_assert(std::is_same_v || std::is_same_v || std::is_same_v); - for (auto & join_clause : join_clauses) + for (auto & expr : expressions) { if constexpr (std::is_same_v) - assert(join_clause.key_names_left.size() == join_clause.key_names_right.size()); + assert(expr.key_names_left.size() == expr.key_names_right.size()); - size_t key_names_size = !std::is_same_v ? join_clause.key_names_left.size() : join_clause.key_names_right.size(); - - for (size_t i = 0; i < key_names_size; ++i) + size_t sz = !std::is_same_v ? expr.key_names_left.size() : expr.key_names_right.size(); + for (size_t i = 0; i < sz; ++i) { - bool should_continue; + bool cont; if constexpr (std::is_same_v) - should_continue = callback(join_clause.key_names_left[i], join_clause.key_names_right[i]); - else if constexpr (std::is_same_v) - should_continue = callback(join_clause.key_names_left[i]); + cont = callback(expr.key_names_left[i], expr.key_names_right[i]); + if constexpr (std::is_same_v) + cont = callback(expr.key_names_left[i]); if constexpr (std::is_same_v) - should_continue = callback(join_clause.key_names_right[i]); + cont = callback(expr.key_names_right[i]); - if (!should_continue) + if (!cont) return false; } } - return true; } @@ -131,7 +133,6 @@ void TableJoin::resetCollected() void TableJoin::addUsingKey(const ASTPtr & ast) { - // std::cout << "TableJoin::addUsingKey " << ast->formatForErrorMessage() << std::endl; addKey(ast->getColumnName(), renamedRightColumnName(ast->getAliasOrColumnName()), ast); } @@ -236,12 +237,6 @@ ASTPtr TableJoin::rightKeysList() const return keys_list; } -void TableJoin::setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix) -{ - columns_from_joined_table = std::move(columns_from_joined_table_value); - deduplicateAndQualifyColumnNames(left_table_columns, right_table_prefix); -} - Names TableJoin::requiredJoinedNames() const { Names key_names_right = getAllNames(JoinTableSide::Right); @@ -261,19 +256,16 @@ Names TableJoin::requiredJoinedNames() const NameSet TableJoin::requiredRightKeys() const { - NameSet required_right_column_names; - forKeyNamesInJoinClauses(clauses, [&](const auto & right_column_name) + NameSet required; + forAllKeys(clauses, [this, &required](const auto & name) { - auto renamed_right_column = renamedRightColumnName(right_column_name); - + auto rename = renamedRightColumnName(name); for (const auto & column : columns_added_by_join) - if (renamed_right_column == column.name) - required_right_column_names.insert(renamed_right_column); - + if (rename == column.name) + required.insert(name); return true; }); - - return required_right_column_names; + return required; } NamesWithAliases TableJoin::getRequiredColumns(const Block & sample, const Names & action_required_columns) const @@ -294,7 +286,7 @@ Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vecto if (required_keys.empty()) return required_right_keys; - forKeyNamesInJoinClauses(clauses, [&](const auto & left_key_name, const auto & right_key_name) + forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) { if (required_keys.contains(right_key_name) && !required_right_keys.has(right_key_name)) { @@ -302,10 +294,8 @@ Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vecto required_right_keys.insert(right_key); keys_sources.push_back(left_key_name); } - return true; }); - return required_right_keys; } @@ -321,15 +311,9 @@ bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column) { - // std::cout << "TableJoin::addJoinedColumn " << joined_column.dump() << std::endl; columns_added_by_join.emplace_back(joined_column); } -void TableJoin::setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value) -{ - columns_added_by_join = columns_added_by_join_value; -} - NamesAndTypesList TableJoin::correctedColumnsAddedByJoin() const { NamesAndTypesList result; @@ -491,7 +475,7 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context) else { /// Can't extract column from dictionary table - /// TODO: Sometimes it should be possible to reconstruct required column, + /// TODO: Sometimes it should be possible to recunstruct required column, /// e.g. if it's an expression depending on dictionary attributes return false; } @@ -558,7 +542,7 @@ TableJoin::createConvertingActions( log_actions("Right", right_converting_actions); } - forKeyNamesInJoinClauses(clauses, [&](auto & left_key, auto & right_key) + forAllKeys(clauses, [&](auto & left_key, auto & right_key) { renameIfNeeded(left_key, left_key_column_rename); renameIfNeeded(right_key, right_key_column_rename); @@ -592,7 +576,7 @@ void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig throw DB::Exception("ASOF join over right table Nullable column is not implemented", ErrorCodes::NOT_IMPLEMENTED); } - forKeyNamesInJoinClauses(clauses, [&](const auto & left_key_name, const auto & right_key_name) + forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) { auto ltypeit = left_types.find(left_key_name); auto rtypeit = right_types.find(right_key_name); @@ -793,7 +777,7 @@ std::unordered_map TableJoin::leftToRightKeyRemap() const if (hasUsing()) { const auto & required_right_keys = requiredRightKeys(); - forKeyNamesInJoinClauses(clauses, [&](const auto & left_key_name, const auto & right_key_name) + forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) { if (!required_right_keys.contains(right_key_name)) left_to_right_key_remap[left_key_name] = right_key_name; @@ -807,16 +791,14 @@ Names TableJoin::getAllNames(JoinTableSide side) const { Names res; auto func = [&res](const auto & name) { res.emplace_back(name); return true; }; - if (side == JoinTableSide::Left) - forKeyNamesInJoinClauses(clauses, func); + forAllKeys(clauses, func); else - forKeyNamesInJoinClauses(clauses, func); - + forAllKeys(clauses, func); return res; } -void TableJoin::assertHasSingleClause() const +void TableJoin::assertHasOneOnExpr() const { if (!oneDisjunct()) { diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index cfa05cb7de1..11675012d0c 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -175,7 +175,7 @@ private: void addKey(const String & left_name, const String & right_name, const ASTPtr & left_ast, const ASTPtr & right_ast = nullptr); - void assertHasSingleClause() const; + void assertHasOneOnExpr() const; /// Calculates common supertypes for corresponding join key columns. template @@ -239,8 +239,8 @@ public: ASTTableJoin & getTableJoin() { return table_join; } const ASTTableJoin & getTableJoin() const { return table_join; } - JoinOnClause & getOnlyClause() { assertHasSingleClause(); return clauses[0]; } - const JoinOnClause & getOnlyClause() const { assertHasSingleClause(); return clauses[0]; } + JoinOnClause & getOnlyClause() { assertHasOneOnExpr(); return clauses[0]; } + const JoinOnClause & getOnlyClause() const { assertHasOneOnExpr(); return clauses[0]; } std::vector & getClauses() { return clauses; } const std::vector & getClauses() const { return clauses; } @@ -282,7 +282,10 @@ public: bool leftBecomeNullable(const DataTypePtr & column_type) const; bool rightBecomeNullable(const DataTypePtr & column_type) const; void addJoinedColumn(const NameAndTypePair & joined_column); - void setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value); + void setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value) + { + columns_added_by_join = columns_added_by_join_value; + } template void addJoinedColumnsAndCorrectTypesImpl(TColumns & left_columns, bool correct_nullability); @@ -304,7 +307,11 @@ public: ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only - void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix); + void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix) + { + columns_from_joined_table = std::move(columns_from_joined_table_value); + deduplicateAndQualifyColumnNames(left_table_columns, right_table_prefix); + } const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; } diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index e16647091ba..48acfb5512a 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -14,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int UNSUPPORTED_METHOD; } bool hasJoin(const ASTSelectQuery & select) @@ -118,6 +119,10 @@ Block getHeaderForProcessingStage( case QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit: case QueryProcessingStage::MAX: { + /// TODO: Analyzer syntax analyzer result + if (!query_info.syntax_analyzer_result) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getHeaderForProcessingStage is unsupported"); + auto query = query_info.query->clone(); TreeRewriterResult new_rewriter_result = *query_info.syntax_analyzer_result; removeJoin(*query->as(), new_rewriter_result, context); diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 8f615b7ce35..4d9de01dc6b 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -14,6 +14,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + namespace { @@ -79,14 +84,14 @@ public: else { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Function IN is supported only if second argument is constant or table expression"); + "Function {} is supported only if second argument is constant or table expression", + function_node->getFunctionName()); } } static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { - return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || - child_node->getNodeType() == QueryTreeNodeType::UNION); + return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); } }; diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 3c073ee03bb..bb6a0dec3a1 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -14,6 +14,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int UNSUPPORTED_METHOD; +} + namespace { @@ -83,7 +89,7 @@ public: static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { - return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; + return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); } }; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 775989cac9c..a31b453718e 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -75,12 +75,7 @@ namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; extern const int LOGICAL_ERROR; - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; - extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; - extern const int INVALID_JOIN_ON_EXPRESSION; - extern const int ILLEGAL_AGGREGATION; - extern const int NOT_AN_AGGREGATE; - extern const int INVALID_WITH_FILL_EXPRESSION; + extern const int BAD_ARGUMENTS; } /** ClickHouse query planner. @@ -102,6 +97,7 @@ namespace ErrorCodes * TODO: Support max streams * TODO: Support ORDER BY read in order optimization * TODO: Support GROUP BY read in order optimization + * TODO: Support Key Condition */ namespace @@ -243,10 +239,12 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, join_tree_node, planner_context); + join_clauses_and_actions.left_join_expressions_actions->projectInput(); auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); left_join_expressions_actions_step->setStepDescription("JOIN actions"); left_plan.addStep(std::move(left_join_expressions_actions_step)); + join_clauses_and_actions.right_join_expressions_actions->projectInput(); auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); right_join_expressions_actions_step->setStepDescription("JOIN actions"); right_plan.addStep(std::move(right_join_expressions_actions_step)); @@ -312,6 +310,7 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, output_node = &cast_actions_dag->addFunction(func_builder_cast, std::move(children), output_node->result_name); } + cast_actions_dag->projectInput(); auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); cast_join_columns_step->setStepDescription("Cast JOIN USING columns"); @@ -339,6 +338,7 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); } + cast_actions_dag->projectInput(); auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable"); plan_to_add_cast.addStep(std::move(cast_join_columns_step)); @@ -478,6 +478,21 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, auto result_plan = QueryPlan(); result_plan.unitePlans(std::move(join_step), {std::move(plans)}); + auto drop_unused_columns_after_join = std::make_shared(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs updated_outputs; + + for (auto & output : drop_unused_columns_after_join->getOutputs()) + { + if (output->result_name.starts_with("__column")) + updated_outputs.push_back(output); + } + + drop_unused_columns_after_join->getOutputs() = std::move(updated_outputs); + + auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentDataStream(), std::move(drop_unused_columns_after_join)); + drop_unused_columns_after_join_transform_step->setStepDescription("DROP unused columns after JOIN"); + result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); + return result_plan; } @@ -512,6 +527,7 @@ QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression, } } + array_join_action_dag->projectInput(); auto array_join_actions = std::make_unique(left_plan.getCurrentDataStream(), array_join_action_dag); array_join_actions->setStepDescription("ARRAY JOIN actions"); left_plan.addStep(std::move(array_join_actions)); @@ -715,6 +731,23 @@ void Planner::buildQueryPlanIfNeeded() auto & query_node = query_tree->as(); + if (query_node.hasPrewhere()) + { + if (query_node.hasWhere()) + { + auto function_node = std::make_shared("and"); + auto and_function = FunctionFactory::instance().get("and", query_context); + function_node->resolveAsFunction(std::move(and_function), std::make_shared()); + function_node->getArguments().getNodes() = {query_node.getPrewhere(), query_node.getWhere()}; + query_node.getWhere() = std::move(function_node); + query_node.getPrewhere() = {}; + } + else + { + query_node.getWhere() = query_node.getPrewhere(); + } + } + SelectQueryInfo select_query_info; select_query_info.original_query = queryNodeToSelectQuery(query_tree); select_query_info.query = select_query_info.original_query; @@ -765,6 +798,7 @@ void Planner::buildQueryPlanIfNeeded() PlannerActionsVisitor actions_visitor(planner_context); GroupingSetsParamsList grouping_sets_parameters_list; bool group_by_with_constant_keys = false; + bool disable_grouping_sets = false; if (query_node.hasGroupBy()) { @@ -818,6 +852,13 @@ void Planner::buildQueryPlanIfNeeded() grouping_sets_parameter.used_keys = std::move(grouping_sets_keys); } + + /// It is expected by execution layer that if there are only 1 grouping sets it will be removed + if (grouping_sets_parameters_list.size() == 1) + { + disable_grouping_sets = true; + grouping_sets_parameters_list.clear(); + } } else { @@ -864,8 +905,8 @@ void Planner::buildQueryPlanIfNeeded() /** For non ordinary GROUP BY we add virtual __grouping_set column * With set number, which is used as an additional key at the stage of merging aggregating data. */ - if (query_node.isGroupByWithRollup() || query_node.isGroupByWithCube() || query_node.isGroupByWithGroupingSets()) - aggregates_columns.emplace_back(nullptr, std::make_shared(), "__grouping_set"); + if (query_node.isGroupByWithRollup() || query_node.isGroupByWithCube() || (query_node.isGroupByWithGroupingSets() && !disable_grouping_sets)) + aggregates_columns.emplace_back(nullptr, std::make_shared(), "__grouping_set"); resolveGroupingFunctions(query_tree, aggregation_keys, grouping_sets_parameters_list, *planner_context); auto aggregate_step = std::make_unique(std::move(group_by_actions_dag), ActionsChainStep::AvailableOutputColumnsStrategy::OUTPUT_NODES, aggregates_columns); @@ -1249,11 +1290,29 @@ void Planner::buildQueryPlanIfNeeded() throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interpolate expression expected to have single action node"); const auto * expression_to_interpolate = expression_to_interpolate_expression_nodes[0]; - const auto & alias_name = expression_to_interpolate->result_name; + const auto & expression_to_interpolate_name = expression_to_interpolate->result_name; const auto * interpolate_expression = interpolate_expression_nodes[0]; - const auto * alias_node = &interpolate_actions_dag->addAlias(*interpolate_expression, alias_name); + if (!interpolate_expression->result_type->equals(*expression_to_interpolate->result_type)) + { + auto cast_type_name = expression_to_interpolate->result_type->getName(); + ColumnWithTypeAndName column; + column.name = "__constant_" + cast_type_name; + column.column = DataTypeString().createColumnConst(0, cast_type_name); + column.type = std::make_shared(); + + const auto * cast_type_constant_node = &interpolate_actions_dag->addColumn(std::move(column)); + + FunctionCastBase::Diagnostic diagnostic = {interpolate_expression->result_name, interpolate_expression->result_name}; + FunctionOverloadResolverPtr func_builder_cast + = CastInternalOverloadResolver::createImpl(std::move(diagnostic)); + + ActionsDAG::NodeRawConstPtrs children = {interpolate_expression, cast_type_constant_node}; + interpolate_expression = &interpolate_actions_dag->addFunction(func_builder_cast, std::move(children), interpolate_expression->result_name); + } + + const auto * alias_node = &interpolate_actions_dag->addAlias(*interpolate_expression, expression_to_interpolate_name); interpolate_actions_dag->getOutputs().push_back(alias_node); } diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 774f0c4babe..4d6d7b3f8d1 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -32,8 +33,7 @@ namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; extern const int LOGICAL_ERROR; - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; - extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; + extern const int BAD_ARGUMENTS; } namespace @@ -514,56 +514,86 @@ String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerConte } case QueryTreeNodeType::FUNCTION: { - const auto & function_node = node->as(); - - WriteBufferFromOwnString buffer; - buffer << "__function_" + function_node.getFunctionName(); - - const auto & function_parameters_nodes = function_node.getParameters().getNodes(); - - if (!function_parameters_nodes.empty()) + if (auto node_constant_value = node->getConstantValueOrNull()) { + result = calculateConstantActionNodeName(node_constant_value->getValue(), node_constant_value->getType()); + } + else + { + const auto & function_node = node->as(); + String in_function_second_argument_node_name; + + if (isNameOfInFunction(function_node.getFunctionName())) + { + const auto & in_second_argument_node = function_node.getArguments().getNodes().at(1); + in_function_second_argument_node_name = planner_context.getGlobalPlannerContext()->getSetKey(in_second_argument_node); + } + + WriteBufferFromOwnString buffer; + buffer << "__function_" + function_node.getFunctionName(); + + const auto & function_parameters_nodes = function_node.getParameters().getNodes(); + + if (!function_parameters_nodes.empty()) + { + buffer << '('; + + size_t function_parameters_nodes_size = function_parameters_nodes.size(); + for (size_t i = 0; i < function_parameters_nodes_size; ++i) + { + const auto & function_parameter_node = function_parameters_nodes[i]; + buffer << calculateActionNodeName(function_parameter_node, planner_context, node_to_name); + + if (i + 1 != function_parameters_nodes_size) + buffer << ", "; + } + + buffer << ')'; + } + + const auto & function_arguments_nodes = function_node.getArguments().getNodes(); + String function_argument_name; + buffer << '('; - size_t function_parameters_nodes_size = function_parameters_nodes.size(); - for (size_t i = 0; i < function_parameters_nodes_size; ++i) + size_t function_arguments_nodes_size = function_arguments_nodes.size(); + for (size_t i = 0; i < function_arguments_nodes_size; ++i) { - const auto & function_parameter_node = function_parameters_nodes[i]; - buffer << calculateActionNodeName(function_parameter_node, planner_context, node_to_name); + if (i == 1 && !in_function_second_argument_node_name.empty()) + { + function_argument_name = in_function_second_argument_node_name; + } + else + { + const auto & function_argument_node = function_arguments_nodes[i]; + function_argument_name = calculateActionNodeName(function_argument_node, planner_context, node_to_name); + } - if (i + 1 != function_parameters_nodes_size) + buffer << function_argument_name; + + if (i + 1 != function_arguments_nodes_size) buffer << ", "; } buffer << ')'; + + result = buffer.str(); } - - const auto & function_arguments_nodes = function_node.getArguments().getNodes(); - - buffer << '('; - - size_t function_arguments_nodes_size = function_arguments_nodes.size(); - for (size_t i = 0; i < function_arguments_nodes_size; ++i) - { - const auto & function_argument_node = function_arguments_nodes[i]; - buffer << calculateActionNodeName(function_argument_node, planner_context, node_to_name); - - if (i + 1 != function_arguments_nodes_size) - buffer << ", "; - } - - buffer << ')'; - - result = buffer.str(); break; } case QueryTreeNodeType::UNION: [[fallthrough]]; case QueryTreeNodeType::QUERY: { - auto query_hash = node->getTreeHash(); - - result = "__subquery_" + std::to_string(query_hash.first) + '_' + std::to_string(query_hash.second); + if (auto node_constant_value = node->getConstantValueOrNull()) + { + result = calculateConstantActionNodeName(node_constant_value->getValue(), node_constant_value->getType()); + } + else + { + auto query_hash = node->getTreeHash(); + result = "__subquery_" + std::to_string(query_hash.first) + '_' + std::to_string(query_hash.second); + } break; } case QueryTreeNodeType::LAMBDA: @@ -575,9 +605,7 @@ String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerConte } default: { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Invalid action query tree node {}", - node->formatASTForErrorMessage()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid action query tree node {}", node->formatASTForErrorMessage()); } } diff --git a/src/Planner/PlannerAggregation.cpp b/src/Planner/PlannerAggregation.cpp index 8e824aa8588..8a306fd464c 100644 --- a/src/Planner/PlannerAggregation.cpp +++ b/src/Planner/PlannerAggregation.cpp @@ -137,7 +137,7 @@ public: static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { - return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; + return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); } }; diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index d8357046646..462abb9e0af 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -16,11 +16,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - /// Subquery node for set struct SubqueryNodeForSet { diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index fb4486a18fe..57b4792d9bc 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -74,6 +74,7 @@ namespace { std::optional extractJoinTableSideFromExpression(const ActionsDAG::Node * expression_root_node, + const std::unordered_set & join_expression_dag_input_nodes, const NameSet & left_table_expression_columns_names, const NameSet & right_table_expression_columns_names, const JoinNode & join_node) @@ -90,7 +91,7 @@ std::optional extractJoinTableSideFromExpression(const ActionsDAG for (const auto & child : node_to_process->children) nodes_to_process.push_back(child); - if (node_to_process->type != ActionsDAG::ActionType::INPUT) + if (!join_expression_dag_input_nodes.contains(node_to_process)) continue; const auto & input_name = node_to_process->result_name; @@ -119,6 +120,7 @@ std::optional extractJoinTableSideFromExpression(const ActionsDAG } void buildJoinClause(ActionsDAGPtr join_expression_dag, + const std::unordered_set & join_expression_dag_input_nodes, const ActionsDAG::Node * join_expressions_actions_node, const NameSet & left_table_expression_columns_names, const NameSet & right_table_expression_columns_names, @@ -131,6 +133,7 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, for (const auto & child : join_expressions_actions_node->children) { buildJoinClause(join_expression_dag, + join_expression_dag_input_nodes, child, left_table_expression_columns_names, right_table_expression_columns_names, @@ -147,11 +150,13 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, const auto * equals_right_child = join_expressions_actions_node->children.at(1); auto left_equals_expression_side_optional = extractJoinTableSideFromExpression(equals_left_child, + join_expression_dag_input_nodes, left_table_expression_columns_names, right_table_expression_columns_names, join_node); auto right_equals_expression_side_optional = extractJoinTableSideFromExpression(equals_right_child, + join_expression_dag_input_nodes, left_table_expression_columns_names, right_table_expression_columns_names, join_node); @@ -199,6 +204,7 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, } auto expression_side_optional = extractJoinTableSideFromExpression(join_expressions_actions_node, + join_expression_dag_input_nodes, left_table_expression_columns_names, right_table_expression_columns_names, join_node); @@ -221,6 +227,18 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & { ActionsDAGPtr join_expression_actions = std::make_shared(join_expression_input_columns); + /** In ActionsDAG if input node has constant representation additional constant column is added. + * That way we cannot simply check that node has INPUT type during resolution of expression join table side. + * Put all nodes after actions dag initialization in set. + * To check if actions dag node is input column, we set contains node. + */ + const auto & join_expression_actions_nodes = join_expression_actions->getNodes(); + + std::unordered_set join_expression_dag_input_nodes; + join_expression_dag_input_nodes.reserve(join_expression_actions_nodes.size()); + for (const auto & node : join_expression_actions_nodes) + join_expression_dag_input_nodes.insert(&node); + PlannerActionsVisitor join_expression_visitor(planner_context); auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_node.getJoinExpression()); if (join_expression_dag_node_raw_pointers.size() != 1) @@ -273,6 +291,7 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & result.join_clauses.emplace_back(); buildJoinClause(join_expression_actions, + join_expression_dag_input_nodes, child, join_left_actions_names_set, join_right_actions_names_set, @@ -285,6 +304,7 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & result.join_clauses.emplace_back(); buildJoinClause(join_expression_actions, + join_expression_dag_input_nodes, join_expressions_actions_root_node, join_left_actions_names_set, join_right_actions_names_set, @@ -421,8 +441,8 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & } JoinClausesAndActions buildJoinClausesAndActions( - const ColumnsWithTypeAndName & left_stream_columns, - const ColumnsWithTypeAndName & right_stream_columns, + const ColumnsWithTypeAndName & left_table_expression_columns, + const ColumnsWithTypeAndName & right_table_expression_columns, const QueryTreeNodePtr & join_node, const PlannerContextPtr & planner_context) { @@ -432,10 +452,10 @@ JoinClausesAndActions buildJoinClausesAndActions( "JOIN {} join does not have ON section", join_node_typed.formatASTForErrorMessage()); - auto join_expression_input_columns = left_stream_columns; - join_expression_input_columns.insert(join_expression_input_columns.end(), right_stream_columns.begin(), right_stream_columns.end()); + auto join_expression_input_columns = left_table_expression_columns; + join_expression_input_columns.insert(join_expression_input_columns.end(), right_table_expression_columns.begin(), right_table_expression_columns.end()); - return buildJoinClausesAndActions(join_expression_input_columns, left_stream_columns, right_stream_columns, join_node_typed, planner_context); + return buildJoinClausesAndActions(join_expression_input_columns, left_table_expression_columns, right_table_expression_columns, join_node_typed, planner_context); } } diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 121a2ea6ce3..3afe6eadb07 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -141,8 +141,8 @@ struct JoinClausesAndActions * planner_context - planner context. */ JoinClausesAndActions buildJoinClausesAndActions( - const ColumnsWithTypeAndName & left_stream_columns, - const ColumnsWithTypeAndName & right_stream_columns, + const ColumnsWithTypeAndName & left_table_expression_columns, + const ColumnsWithTypeAndName & right_table_expression_columns, const QueryTreeNodePtr & join_node, const PlannerContextPtr & planner_context); diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 6ce43097d1c..138f24a883a 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -22,6 +22,7 @@ namespace DB namespace ErrorCodes { extern const int TYPE_MISMATCH; + extern const int LOGICAL_ERROR; } String dumpQueryPlan(QueryPlan & query_plan) diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index a8431d38a78..e3eaed358f6 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -70,6 +70,10 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, const auto & query_info = read_from_merge_tree->getQueryInfo(); const auto * select_query = query_info.query->as(); + /// TODO: Analyzer syntax analyzer result + if (!query_info.syntax_analyzer_result) + return 0; + ManyExpressionActions order_by_elements_actions; const auto & window_desc = window->getWindowDescription(); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 4e73323ee89..b282fb79514 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -889,6 +889,12 @@ KeyCondition::KeyCondition( key_columns[name] = i; } + if (!syntax_analyzer_result) + { + rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); + return; + } + /** Evaluation of expressions that depend only on constants. * For the index to be used, if it is written, for example `WHERE Date = toDate(now())`. */ @@ -969,12 +975,15 @@ KeyCondition::KeyCondition( key_columns[name] = i; } - if (syntax_analyzer_result) + if (!syntax_analyzer_result) { - for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source) - array_joined_columns.insert(name); + rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); + return; } + for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source) + array_joined_columns.insert(name); + if (!dag_nodes.nodes.empty()) { auto inverted_dag = cloneASTWithInversionPushDown(std::move(dag_nodes.nodes), context); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 66950734d5f..510e8d0da84 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5702,6 +5702,11 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg { const auto & metadata_snapshot = storage_snapshot->metadata; const auto & settings = query_context->getSettingsRef(); + + /// TODO: Analyzer syntax analyzer result + if (!query_info.syntax_analyzer_result) + return std::nullopt; + if (!settings.allow_experimental_projection_optimization || query_info.ignore_projections || query_info.is_projection_query || settings.aggregate_functions_null_for_empty /* projections don't work correctly with this setting */) return std::nullopt; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index bced552915b..8e4715db483 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -564,6 +564,10 @@ std::optional StorageDistributed::getOptimizedQueryP return {}; } + /// TODO: Analyzer syntax analyzer result + if (!query_info.syntax_analyzer_result) + return {}; + // GROUP BY const ASTPtr group_by = select.groupBy(); if (!query_info.syntax_analyzer_result->aggregates.empty() || group_by) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index dc94b6cee98..7fb21b7e053 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -225,11 +225,15 @@ SelectQueryInfo getModifiedQueryInfo( SelectQueryInfo modified_query_info = query_info; modified_query_info.query = query_info.query->clone(); - /// Original query could contain JOIN but we need only the first joined table and its columns. - auto & modified_select = modified_query_info.query->as(); - TreeRewriterResult new_analyzer_res = *modified_query_info.syntax_analyzer_result; - removeJoin(modified_select, new_analyzer_res, modified_context); - modified_query_info.syntax_analyzer_result = std::make_shared(std::move(new_analyzer_res)); + /// TODO: Analyzer syntax analyzer result + if (modified_query_info.syntax_analyzer_result) + { + /// Original query could contain JOIN but we need only the first joined table and its columns. + auto & modified_select = modified_query_info.query->as(); + TreeRewriterResult new_analyzer_res = *modified_query_info.syntax_analyzer_result; + removeJoin(modified_select, new_analyzer_res, modified_context); + modified_query_info.syntax_analyzer_result = std::make_shared(std::move(new_analyzer_res)); + } if (!is_merge_engine) { diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index c42fb7fa965..51b11680f82 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int INCORRECT_QUERY; + extern const int UNSUPPORTED_METHOD; } namespace @@ -251,6 +252,11 @@ String transformQueryForExternalDatabase( ContextPtr context) { auto clone_query = query_info.query->clone(); + + /// TODO: Analyzer syntax analyzer result + if (!query_info.syntax_analyzer_result) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "transform query for external database is unsupported"); + const Names used_columns = query_info.syntax_analyzer_result->requiredSourceColumns(); bool strict = context->getSettingsRef().external_table_strict_query; diff --git a/tests/queries/0_stateless/02376_analyzer_in_function_subquery.reference b/tests/queries/0_stateless/02376_analyzer_in_function_subquery.reference index 4ae0e409261..c0bb30b101f 100644 --- a/tests/queries/0_stateless/02376_analyzer_in_function_subquery.reference +++ b/tests/queries/0_stateless/02376_analyzer_in_function_subquery.reference @@ -6,12 +6,6 @@ SELECT '--'; -- SELECT id, value FROM test_table WHERE id IN (SELECT 2); 2 Value_2 -DROP TABLE IF EXISTS test_table_for_in; -CREATE TABLE test_table_for_in -( - id UInt64 -) ENGINE=TinyLog; -INSERT INTO test_table_for_in VALUES (0), (1); SELECT '--'; -- SELECT id, value FROM test_table WHERE id IN test_table_for_in; diff --git a/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql b/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql index b199de96534..65b64f8a63a 100644 --- a/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql +++ b/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql @@ -9,14 +9,6 @@ CREATE TABLE test_table INSERT INTO test_table VALUES (0, 'Value_0'), (1, 'Value_1'), (2, 'Value_2'); --- { echoOn } - -SELECT id, value FROM test_table WHERE id IN (SELECT 1); - -SELECT '--'; - -SELECT id, value FROM test_table WHERE id IN (SELECT 2); - DROP TABLE IF EXISTS test_table_for_in; CREATE TABLE test_table_for_in ( @@ -25,6 +17,14 @@ CREATE TABLE test_table_for_in INSERT INTO test_table_for_in VALUES (0), (1); +-- { echoOn } + +SELECT id, value FROM test_table WHERE id IN (SELECT 1); + +SELECT '--'; + +SELECT id, value FROM test_table WHERE id IN (SELECT 2); + SELECT '--'; SELECT id, value FROM test_table WHERE id IN test_table_for_in; From 4f68305dbe4ae64f80d863f7d27830f4c34b3880 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 6 Sep 2022 18:46:30 +0200 Subject: [PATCH 097/188] Fixed tests --- src/Analyzer/ArrayJoinNode.cpp | 2 +- src/Analyzer/ColumnTransformers.h | 4 +- src/Analyzer/FunctionNode.h | 6 +- src/Analyzer/IQueryTreeNode.h | 2 +- src/Analyzer/Identifier.h | 8 +- src/Analyzer/JoinNode.cpp | 4 +- src/Analyzer/MatcherNode.h | 2 +- src/Analyzer/QueryAnalysisPass.cpp | 26 +++--- src/Analyzer/QueryNode.cpp | 2 +- src/Analyzer/QueryTreeBuilder.cpp | 14 ++-- src/Analyzer/UnionNode.cpp | 41 +++++---- src/Interpreters/PreparedSets.h | 1 - src/Interpreters/TableJoin.cpp | 69 --------------- src/Parsers/SelectUnionMode.cpp | 28 ++++--- src/Planner/Planner.cpp | 84 ++++++++++++------- src/Planner/PlannerActionsVisitor.h | 1 - src/Planner/PlannerContext.h | 3 +- src/Planner/PlannerJoins.h | 4 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 15 ++-- src/Processors/QueryPlan/CreatingSetsStep.h | 2 + 20 files changed, 153 insertions(+), 165 deletions(-) diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp index d88164230e5..20eb3f7f074 100644 --- a/src/Analyzer/ArrayJoinNode.cpp +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -27,7 +27,7 @@ void ArrayJoinNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_stat buffer << '\n' << std::string(indent + 2, ' ') << "TABLE EXPRESSION\n"; getTableExpression()->dumpTreeImpl(buffer, format_state, indent + 4); - buffer << '\n' << std::string(indent + 2, ' ') << "JOIN EXPRESSSIONS\n"; + buffer << '\n' << std::string(indent + 2, ' ') << "JOIN EXPRESSIONS\n"; getJoinExpressionsNode()->dumpTreeImpl(buffer, format_state, indent + 4); } diff --git a/src/Analyzer/ColumnTransformers.h b/src/Analyzer/ColumnTransformers.h index 5956824ebd7..f32ec1b96cf 100644 --- a/src/Analyzer/ColumnTransformers.h +++ b/src/Analyzer/ColumnTransformers.h @@ -27,7 +27,7 @@ namespace DB * 1. regexp variant: SELECT matcher EXCEPT ('regexp'). * 2. column names list variant: SELECT matcher EXCEPT (column_name_1, ...). * - * 3. REPLACE transfomer: + * 3. REPLACE transformer: * REPLACE transformer applies similar transformation as APPLY transformer, but only for expressions * that match replacement expression name. * @@ -243,7 +243,7 @@ using ReplaceColumnTransformerNodePtr = std::shared_ptr; /** Function node represents function in query tree. * Function syntax: function_name(parameter_1, ...)(argument_1, ...). - * If funciton does not have parameters its syntax is function_name(argument_1, ...). + * If function does not have parameters its syntax is function_name(argument_1, ...). * If function does not have arguments its syntax is function_name(). * * In query tree function parameters and arguments are represented by ListNode. @@ -130,7 +130,7 @@ public: /** Resolve function node as non aggregate function. * It is important that function name is update with resolved function name. - * Main motiviation for this is query tree optimizations. + * Main motivation for this is query tree optimizations. * Assume we have `multiIf` function with single argument, it can be converted to `if` function. * Function name must be updated accordingly. */ @@ -138,7 +138,7 @@ public: /** Resolve function node as aggregate function. * It is important that function name is update with resolved function name. - * Main motiviation for this is query tree optimizations. + * Main motivation for this is query tree optimizations. */ void resolveAsAggregateFunction(AggregateFunctionPtr aggregate_function_value, DataTypePtr result_type_value); diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 0bfd73085df..58762411025 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -80,7 +80,7 @@ public: } /** Get result type of query tree node that can be used as part of expression. - * If node does not support this method exception is throwed. + * If node does not support this method exception is thrown. * TODO: Maybe this can be a part of ExpressionQueryTreeNode. */ virtual DataTypePtr getResultType() const diff --git a/src/Analyzer/Identifier.h b/src/Analyzer/Identifier.h index d799060f8cf..9d70d4dfc41 100644 --- a/src/Analyzer/Identifier.h +++ b/src/Analyzer/Identifier.h @@ -10,7 +10,7 @@ namespace DB { -/** Identifier constists from identifier parts. +/** Identifier consists from identifier parts. * Each identifier part is arbitrary long sequence of digits, underscores, lowercase and uppercase letters. * Example: a, a.b, a.b.c. */ @@ -26,21 +26,21 @@ public: { } - /// Create Identifier from parts + /// Create Identifier from parts explicit Identifier(std::vector && parts_) : parts(std::move(parts_)) , full_name(boost::algorithm::join(parts, ".")) { } - /// Create Identifier from full name, full name is splitted with '.' as separator. + /// Create Identifier from full name, full name is split with '.' as separator. explicit Identifier(const std::string & full_name_) : full_name(full_name_) { boost::split(parts, full_name, [](char c) { return c == '.'; }); } - /// Create Identifier from full name, full name is splitted with '.' as separator. + /// Create Identifier from full name, full name is split with '.' as separator. explicit Identifier(std::string && full_name_) : full_name(std::move(full_name_)) { diff --git a/src/Analyzer/JoinNode.cpp b/src/Analyzer/JoinNode.cpp index fcf2bf4c8d5..cd006efdfd3 100644 --- a/src/Analyzer/JoinNode.cpp +++ b/src/Analyzer/JoinNode.cpp @@ -70,12 +70,12 @@ void JoinNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, si buffer << '\n' << std::string(indent + 2, ' ') << "LEFT TABLE EXPRESSION\n"; getLeftTableExpression()->dumpTreeImpl(buffer, format_state, indent + 4); - buffer << '\n' << std::string(indent + 2, ' ') << "RIGHT TABLE EXPRESSSION\n"; + buffer << '\n' << std::string(indent + 2, ' ') << "RIGHT TABLE EXPRESSION\n"; getRightTableExpression()->dumpTreeImpl(buffer, format_state, indent + 4); if (getJoinExpression()) { - buffer << '\n' << std::string(indent + 2, ' ') << "JOIN EXPRESSSION\n"; + buffer << '\n' << std::string(indent + 2, ' ') << "JOIN EXPRESSION\n"; getJoinExpression()->dumpTreeImpl(buffer, format_state, indent + 4); } } diff --git a/src/Analyzer/MatcherNode.h b/src/Analyzer/MatcherNode.h index 010d6e44e23..ebbde9cd4fd 100644 --- a/src/Analyzer/MatcherNode.h +++ b/src/Analyzer/MatcherNode.h @@ -42,7 +42,7 @@ namespace DB * Example: SELECT test_table.* FROM test_table. * Example: SELECT a.* FROM test_table AS a. * - * Additionaly each matcher can contain transformers, check ColumnTransformers.h. + * Additionally each matcher can contain transformers, check ColumnTransformers.h. * In query tree matchers column transformers are represended as ListNode. */ enum class MatcherNodeType diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index b4eebf289a7..53c1530b761 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -142,7 +142,7 @@ namespace ErrorCodes * 1. Try to resolve identifier in expression context. * 2. Try to resolve identifier in function context, if it is allowed. Example: SELECT func(arguments); Here func identifier cannot be resolved in function context * because query projection does not support that. - * 3. Try to resolve identifier in talbe context, if it is allowed. Example: SELECT table; Here table identifier cannot be resolved in function context + * 3. Try to resolve identifier in table context, if it is allowed. Example: SELECT table; Here table identifier cannot be resolved in function context * because query projection does not support that. * * TODO: This does not supported properly before, because matchers could not be resolved from aliases. @@ -167,13 +167,13 @@ namespace ErrorCodes * * Additional rules about identifier binding. * Bind for identifier to entity means that identifier first part match some node during analysis. - * If other parts of identifier cannot be resolved in that node, exception must be throwed. + * If other parts of identifier cannot be resolved in that node, exception must be thrown. * * Example: * CREATE TABLE test_table (id UInt64, compound_value Tuple(value UInt64)) ENGINE=TinyLog; * SELECT compound_value.value, 1 AS compound_value FROM test_table; * Identifier first part compound_value bound to entity with alias compound_value, but nested identifier part cannot be resolved from entity, - * lookup should not be continued, and exception must be throwed because if lookup continues that way identifier can be resolved from join tree. + * lookup should not be continued, and exception must be thrown because if lookup continues that way identifier can be resolved from join tree. * * TODO: This was not supported properly before analyzer because nested identifier could not be resolved from alias. * @@ -181,7 +181,7 @@ namespace ErrorCodes * CREATE TABLE test_table (id UInt64, value UInt64) ENGINE=TinyLog; * WITH cast(('Value'), 'Tuple (value UInt64') AS value SELECT (SELECT value FROM test_table); * Identifier first part value bound to test_table column value, but nested identifier part cannot be resolved from it, - * lookup should not be continued, and exception must be throwed because if lookup continues identifier can be resolved from parent scope. + * lookup should not be continued, and exception must be thrown because if lookup continues identifier can be resolved from parent scope. * * TODO: Add expression name into query tree node. Example: SELECT plus(1, 1). Result: SELECT 2. Expression name of constant node should be 2. * TODO: Update exception messages @@ -366,7 +366,7 @@ struct IdentifierResolveSettings /// Allow to check join tree during identifier resolution bool allow_to_check_join_tree = true; - /// Allow to check CTEs during table identifer resolution + /// Allow to check CTEs during table identifier resolution bool allow_to_check_cte = true; /// Allow to check database catalog during table identifier resolution @@ -1125,7 +1125,7 @@ void QueryAnalyzer::validateTableExpressionModifiers(QueryTreeNodePtr & table_ex } throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Table expresion modifiers {} are not supported for subquery {}. In scope {}", + "Table expression modifiers {} are not supported for subquery {}. In scope {}", table_expression_modifiers_error_message, table_expression_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); @@ -1679,7 +1679,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo { if (left_resolved_identifier && right_resolved_identifier) throw Exception(ErrorCodes::AMBIGUOUS_IDENTIFIER, - "JOIN {} ambigious identifier {}. In scope {}", + "JOIN {} ambiguous identifier {}. In scope {}", table_expression_node->formatASTForErrorMessage(), identifier_lookup.dump(), scope.scope_node->formatASTForErrorMessage()); @@ -1737,7 +1737,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo else { throw Exception(ErrorCodes::AMBIGUOUS_IDENTIFIER, - "JOIN {} ambigious identifier {}. In scope {}", + "JOIN {} ambiguous identifier {}. In scope {}", table_expression_node->formatASTForErrorMessage(), identifier_lookup.dump(), scope.scope_node->formatASTForErrorMessage()); @@ -1878,7 +1878,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTreeNode(const Ident * 2. Try to resolve identifier from table columns. * 3. If there is no FROM section return nullptr. * 4. If identifier is in table lookup context, check if it has 1 or 2 parts, otherwise throw exception. - * If identifer has 2 parts try to match it with database_name and table_name. + * If identifier has 2 parts try to match it with database_name and table_name. * If identifier has 1 part try to match it with table_name, then try to match it with table alias. * 5. If identifier is in expression lookup context, we first need to bind identifier to some table column using identifier first part. * Start with identifier first part, if it match some column name in table try to get column with full identifier name. @@ -2005,7 +2005,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const * in database catalog. * * Same is not done for functions because function resolution is more complex, and in case of aggregate functions requires not only name - * but also argument types, it is responsiblity of resolve function method to handle resolution of function name. + * but also argument types, it is responsibility of resolve function method to handle resolution of function name. * * 9. If identifier was not resolved remove it from identifier_lookup_to_resolve_status table. * @@ -2596,7 +2596,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, * scope - lambda scope. It is client responsibility to create it. * * Resolve steps: - * 1. Valide arguments. + * 1. Validate arguments. * 2. Register lambda in lambdas in resolve process. This is necessary to prevent recursive lambda resolving. * 3. Initialize scope with lambda aliases. * 4. Validate lambda argument names, and scope expressions. @@ -2966,7 +2966,7 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc } /** For lambda arguments we need to initialize lambda argument types DataTypeFunction using `getLambdaArgumentTypes` function. - * Then each lambda arguments are initalized with columns, where column source is lambda. + * Then each lambda arguments are initialized with columns, where column source is lambda. * This information is important for later steps of query processing. * Example: SELECT arrayMap(x -> x + 1, [1, 2, 3]). * lambda node x -> x + 1 identifier x is resolved as column where source is lambda node. @@ -3122,7 +3122,7 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc * 2. Call specific resolve method depending on node type. * * If allow_table_expression = true and node is query node, then it is not evaluated as scalar subquery. - * Althought if node is identifier that is resolved into query node that query is evaluated as scalar subquery. + * Although if node is identifier that is resolved into query node that query is evaluated as scalar subquery. * SELECT id, (SELECT 1) AS c FROM test_table WHERE a IN c; * SELECT id, FROM test_table WHERE a IN (SELECT 1); * diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 5c27b01081d..64f3a4561ab 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -361,7 +361,7 @@ ASTPtr QueryNode::toASTImpl() const select_query->setExpression(ASTSelectQuery::Expression::LIMIT_OFFSET, getOffset()->toAST()); auto result_select_query = std::make_shared(); - result_select_query->union_mode = SelectUnionMode::Unspecified; + result_select_query->union_mode = SelectUnionMode::UNION_DEFAULT; auto list_of_selects = std::make_shared(); list_of_selects->children.push_back(std::move(select_query)); diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 1e46a6083b8..49ad3ea805f 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -166,10 +166,14 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectIntersectExceptQuery(const ASTPtr union_node->setIsSubquery(is_subquery); union_node->setCTEName(cte_name); - if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::INTERSECT) - union_node->setUnionMode(SelectUnionMode::INTERSECT); - else if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT) - union_node->setUnionMode(SelectUnionMode::EXCEPT); + if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::INTERSECT_ALL) + union_node->setUnionMode(SelectUnionMode::INTERSECT_ALL); + else if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::INTERSECT_DISTINCT) + union_node->setUnionMode(SelectUnionMode::INTERSECT_DISTINCT); + else if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT_ALL) + union_node->setUnionMode(SelectUnionMode::EXCEPT_ALL); + else if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT_DISTINCT) + union_node->setUnionMode(SelectUnionMode::EXCEPT_DISTINCT); else throw Exception(ErrorCodes::LOGICAL_ERROR, "UNION type is not initialized"); @@ -523,7 +527,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select { if (!tables_in_select_query) { - /** If no table is specified in SELECT query we substitude system.one table. + /** If no table is specified in SELECT query we substitute system.one table. * SELECT * FROM system.one; */ Identifier storage_identifier("system.one"); diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 8ff26664711..4e67fa5b169 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -99,10 +99,25 @@ String UnionNode::getName() const continue; auto query_union_mode = union_modes.at(i - 1); - if (query_union_mode == SelectUnionMode::ALL || query_union_mode == SelectUnionMode::DISTINCT) - buffer << " UNION " << toString(query_union_mode); - else - buffer << toString(query_union_mode); + + if (query_union_mode == SelectUnionMode::UNION_DEFAULT) + buffer << "UNION"; + else if (query_union_mode == SelectUnionMode::UNION_ALL) + buffer << "UNION ALL"; + else if (query_union_mode == SelectUnionMode::UNION_DISTINCT) + buffer << "UNION DISTINCT"; + else if (query_union_mode == SelectUnionMode::EXCEPT_DEFAULT) + buffer << "EXCEPT"; + else if (query_union_mode == SelectUnionMode::EXCEPT_ALL) + buffer << "EXCEPT ALL"; + else if (query_union_mode == SelectUnionMode::EXCEPT_DISTINCT) + buffer << "EXCEPT DISTINCT"; + else if (query_union_mode == SelectUnionMode::INTERSECT_DEFAULT) + buffer << "INTERSECT"; + else if (query_union_mode == SelectUnionMode::INTERSECT_ALL) + buffer << "INTERSECT ALL"; + else if (query_union_mode == SelectUnionMode::INTERSECT_DISTINCT) + buffer << "INTERSECT DISTINCT"; } return buffer.str(); @@ -133,12 +148,7 @@ void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s table_expression_modifiers->dump(buffer); } - buffer << ", union_mode: "; - - if (union_mode == SelectUnionMode::ALL || union_mode == SelectUnionMode::DISTINCT) - buffer << " UNION " << toString(union_mode); - else - buffer << toString(union_mode); + buffer << ", union_mode: " << toString(union_mode); size_t union_modes_size = union_modes.size(); buffer << '\n' << std::string(indent + 2, ' ') << "UNION MODES " << union_modes_size << '\n'; @@ -148,10 +158,7 @@ void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << std::string(indent + 4, ' '); auto query_union_mode = union_modes[i]; - if (query_union_mode == SelectUnionMode::ALL || query_union_mode == SelectUnionMode::DISTINCT) - buffer << " UNION " << toString(query_union_mode); - else - buffer << toString(query_union_mode); + buffer << toString(query_union_mode); if (i + 1 != union_modes_size) buffer << '\n'; @@ -208,6 +215,12 @@ ASTPtr UnionNode::toASTImpl() const { auto select_with_union_query = std::make_shared(); select_with_union_query->union_mode = union_mode; + + if (union_mode != SelectUnionMode::UNION_DEFAULT && + union_mode != SelectUnionMode::EXCEPT_DEFAULT && + union_mode != SelectUnionMode::EXCEPT_DEFAULT) + select_with_union_query->is_normalized = true; + select_with_union_query->list_of_modes = union_modes; select_with_union_query->set_of_modes = union_modes_set; select_with_union_query->children.push_back(getQueriesNode()->toAST()); diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 06600c49f13..a50e390ee5a 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -39,7 +39,6 @@ public: /// This is a temporary table for transferring to remote servers for distributed query processing. StoragePtr table; -private: /// The source is obtained using the InterpreterSelectQuery subquery. std::unique_ptr source; }; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 662af320ff9..5d065e564b2 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -416,75 +416,6 @@ bool TableJoin::needStreamWithNonJoinedRows() const return isRightOrFull(kind()); } -static std::optional getDictKeyName(const String & dict_name , ContextPtr context) -{ - auto dictionary = context->getExternalDictionariesLoader().getDictionary(dict_name, context); - if (!dictionary) - return {}; - - if (const auto & structure = dictionary->getStructure(); structure.id) - return structure.id->name; - return {}; -} - -bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context) -{ - bool allowed_inner = isInner(kind()) && strictness() == JoinStrictness::All; - bool allowed_left = isLeft(kind()) && (strictness() == JoinStrictness::Any || - strictness() == JoinStrictness::All || - strictness() == JoinStrictness::Semi || - strictness() == JoinStrictness::Anti); - - /// Support ALL INNER, [ANY | ALL | SEMI | ANTI] LEFT - if (!allowed_inner && !allowed_left) - return false; - - if (clauses.size() != 1 || clauses[0].key_names_right.size() != 1) - return false; - - const auto & right_key = getOnlyClause().key_names_right[0]; - - /// TODO: support 'JOIN ... ON expr(dict_key) = table_key' - auto it_key = original_names.find(right_key); - if (it_key == original_names.end()) - return false; - - if (!right_storage_dictionary) - return false; - - auto dict_name = right_storage_dictionary->getDictionaryName(); - - auto dict_key = getDictKeyName(dict_name, context); - if (!dict_key.has_value() || *dict_key != it_key->second) - return false; /// JOIN key != Dictionary key - - Names src_names; - NamesAndTypesList dst_columns; - for (const auto & col : sample_block) - { - if (col.name == right_key) - continue; /// do not extract key column - - auto it = original_names.find(col.name); - if (it != original_names.end()) - { - String original = it->second; - src_names.push_back(original); - dst_columns.push_back({col.name, col.type}); - } - else - { - /// Can't extract column from dictionary table - /// TODO: Sometimes it should be possible to recunstruct required column, - /// e.g. if it's an expression depending on dictionary attributes - return false; - } - } - dictionary_reader = std::make_shared(dict_name, src_names, dst_columns, context); - - return true; -} - static void renameIfNeeded(String & name, const NameToNameMap & renames) { if (const auto it = renames.find(name); it != renames.end()) diff --git a/src/Parsers/SelectUnionMode.cpp b/src/Parsers/SelectUnionMode.cpp index 4a7fac0fca1..6d56a2b219f 100644 --- a/src/Parsers/SelectUnionMode.cpp +++ b/src/Parsers/SelectUnionMode.cpp @@ -8,16 +8,24 @@ const char * toString(SelectUnionMode mode) { switch (mode) { - case SelectUnionMode::ALL: - return "ALL"; - case SelectUnionMode::DISTINCT: - return "DISTINCT"; - case SelectUnionMode::EXCEPT: - return "EXCEPT"; - case SelectUnionMode::INTERSECT: - return "INTERSECT"; - case SelectUnionMode::Unspecified: - return "Unspecified"; + case SelectUnionMode::UNION_DEFAULT: + return "UNION_DEFAULT"; + case SelectUnionMode::UNION_ALL: + return "UNION_ALL"; + case SelectUnionMode::UNION_DISTINCT: + return "UNION_DISTINCT"; + case SelectUnionMode::EXCEPT_DEFAULT: + return "EXCEPT_DEFAULT"; + case SelectUnionMode::EXCEPT_ALL: + return "EXCEPT_ALL"; + case SelectUnionMode::EXCEPT_DISTINCT: + return "EXCEPT_DISTINCT"; + case SelectUnionMode::INTERSECT_DEFAULT: + return "INTERSECT_DEFAULT"; + case SelectUnionMode::INTERSECT_ALL: + return "INTERSECT_ALL"; + case SelectUnionMode::INTERSECT_DISTINCT: + return "INTERSECT_DEFAULT"; } } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index a31b453718e..ca5b0b8d010 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -93,7 +93,6 @@ namespace ErrorCodes * TODO: Support projections * TODO: Support read in order optimization * TODO: UNION storage limits - * TODO: Interpreter resources * TODO: Support max streams * TODO: Support ORDER BY read in order optimization * TODO: Support GROUP BY read in order optimization @@ -581,7 +580,7 @@ void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, const SelectQ if (select_query_options.is_subquery) return; - SubqueriesForSets subqueries_for_sets; + PreparedSets::SubqueriesForSets subqueries_for_sets; const auto & subquery_node_to_sets = planner_context->getGlobalPlannerContext()->getSubqueryNodesForSets(); for (auto [key, subquery_node_for_set] : subquery_node_to_sets) @@ -602,9 +601,7 @@ void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, const SelectQ subqueries_for_sets.emplace(key, std::move(subquery_for_set)); } - const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); - SizeLimits limits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode); - addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), limits, planner_context->getQueryContext()); + addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); } } @@ -649,7 +646,9 @@ void Planner::buildQueryPlanIfNeeded() if (auto * union_query_tree = query_tree->as()) { auto union_mode = union_query_tree->getUnionMode(); - if (union_mode == SelectUnionMode::Unspecified) + if (union_mode == SelectUnionMode::UNION_DEFAULT || + union_mode == SelectUnionMode::EXCEPT_DEFAULT || + union_mode == SelectUnionMode::INTERSECT_DEFAULT) throw Exception(ErrorCodes::LOGICAL_ERROR, "UNION mode must be initialized"); size_t queries_size = union_query_tree->getQueries().getNodes().size(); @@ -695,37 +694,48 @@ void Planner::buildQueryPlanIfNeeded() const auto & settings = query_context->getSettingsRef(); auto max_threads = settings.max_threads; - if (union_mode == SelectUnionMode::ALL || union_mode == SelectUnionMode::DISTINCT) + bool is_distinct = union_mode == SelectUnionMode::UNION_DISTINCT || union_mode == SelectUnionMode::INTERSECT_DISTINCT || + union_mode == SelectUnionMode::EXCEPT_DISTINCT; + + if (union_mode == SelectUnionMode::UNION_ALL || union_mode == SelectUnionMode::UNION_DISTINCT) { auto union_step = std::make_unique(std::move(query_plans_streams), max_threads); query_plan.unitePlans(std::move(union_step), std::move(query_plans)); - - if (union_query_tree->getUnionMode() == SelectUnionMode::DISTINCT) - { - /// Add distinct transform - SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); - - auto distinct_step = std::make_unique( - query_plan.getCurrentDataStream(), - limits, - 0 /*limit hint*/, - query_plan.getCurrentDataStream().header.getNames(), - false /*pre distinct*/, - settings.optimize_distinct_in_order); - - query_plan.addStep(std::move(distinct_step)); - } } - else if (union_mode == SelectUnionMode::INTERSECT || union_mode == SelectUnionMode::EXCEPT) + else if (union_mode == SelectUnionMode::INTERSECT_ALL || union_mode == SelectUnionMode::INTERSECT_DISTINCT || + union_mode == SelectUnionMode::EXCEPT_ALL || union_mode == SelectUnionMode::EXCEPT_DISTINCT) { - IntersectOrExceptStep::Operator intersect_or_except_operator = IntersectOrExceptStep::Operator::INTERSECT; - if (union_mode == SelectUnionMode::EXCEPT) - intersect_or_except_operator = IntersectOrExceptStep::Operator::EXCEPT; + IntersectOrExceptStep::Operator intersect_or_except_operator = IntersectOrExceptStep::Operator::UNKNOWN; + + if (union_mode == SelectUnionMode::INTERSECT_ALL) + intersect_or_except_operator = IntersectOrExceptStep::Operator::INTERSECT_ALL; + else if (union_mode == SelectUnionMode::INTERSECT_DISTINCT) + intersect_or_except_operator = IntersectOrExceptStep::Operator::INTERSECT_DISTINCT; + else if (union_mode == SelectUnionMode::EXCEPT_ALL) + intersect_or_except_operator = IntersectOrExceptStep::Operator::EXCEPT_ALL; + else if (union_mode == SelectUnionMode::EXCEPT_DISTINCT) + intersect_or_except_operator = IntersectOrExceptStep::Operator::EXCEPT_DISTINCT; auto union_step = std::make_unique(std::move(query_plans_streams), intersect_or_except_operator, max_threads); query_plan.unitePlans(std::move(union_step), std::move(query_plans)); } + if (is_distinct) + { + /// Add distinct transform + SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); + + auto distinct_step = std::make_unique( + query_plan.getCurrentDataStream(), + limits, + 0 /*limit hint*/, + query_plan.getCurrentDataStream().header.getNames(), + false /*pre distinct*/, + settings.optimize_distinct_in_order); + + query_plan.addStep(std::move(distinct_step)); + } + return; } @@ -1082,6 +1092,7 @@ void Planner::buildQueryPlanIfNeeded() settings.min_free_disk_space_for_temporary_data, settings.compile_aggregate_expressions, settings.min_count_to_compile_aggregate_expression, + settings.max_block_size, /* only_merge */ false, stats_collecting_params ); @@ -1237,7 +1248,8 @@ void Planner::buildQueryPlanIfNeeded() settings.remerge_sort_lowered_memory_bytes_ratio, settings.max_bytes_before_external_sort, planner_context->getQueryContext()->getTemporaryVolume(), - settings.min_free_disk_space_for_temporary_data); + settings.min_free_disk_space_for_temporary_data, + settings.optimize_sorting_by_input_stream_properties); sorting_step->setStepDescription("Sorting for ORDER BY"); query_plan.addStep(std::move(sorting_step)); @@ -1411,6 +1423,22 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(projection_step)); addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context); + + /// Extend lifetime of context, table locks, storages + query_plan.addInterpreterContext(planner_context->getQueryContext()); + + for (auto & [table_expression, _] : planner_context->getTableExpressionNodeToData()) + { + if (auto * table_node = table_expression->as()) + { + query_plan.addStorageHolder(table_node->getStorage()); + query_plan.addTableLock(table_node->getStorageLock()); + } + else if (auto * table_function_node = table_expression->as()) + { + query_plan.addStorageHolder(table_function_node->getStorage()); + } + } } } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 746839b4dc8..e699220b0b5 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -6,7 +6,6 @@ #include #include -#include #include diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 462abb9e0af..0dd17a4639f 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -6,7 +6,6 @@ #include #include -#include #include #include @@ -46,7 +45,7 @@ public: /// Get set for key, if no set is registered null is returned SetPtr getSetOrNull(const SetKey & key) const; - /// Get set for key, if no set is registered logical exception is throwed + /// Get set for key, if no set is registered logical exception is thrown SetPtr getSetOrThrow(const SetKey & key) const; /** Register subquery node for set diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 3afe6eadb07..c8f0cb7aa42 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -11,7 +11,7 @@ namespace DB { /** Join clause represent single JOIN ON section clause. - * Join clause consits of JOIN keys and conditions. + * Join clause consists of JOIN keys and conditions. * * JOIN can contain multiple clauses in JOIN ON section. * Example: SELECT * FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id OR t1.value = t2.value; @@ -34,7 +34,7 @@ namespace DB * * We have 2 streams, left stream and right stream. * We split JOIN ON section expressions actions in two parts left join expression actions and right join expression actions. - * Left join expresion actions must be used to calculate necessary actions for left stream. + * Left join expression actions must be used to calculate necessary actions for left stream. * Right join expression actions must be used to calculate necessary actions for right stream. */ class PlannerContext; diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index bd079c0b8a9..b52d86aa725 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -122,11 +122,8 @@ void CreatingSetsStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } -void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context) +void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context) { - if (!prepared_sets || prepared_sets->empty()) - return; - DataStreams input_streams; input_streams.emplace_back(query_plan.getCurrentDataStream()); @@ -134,7 +131,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, plans.emplace_back(std::make_unique(std::move(query_plan))); query_plan = QueryPlan(); - for (auto & [description, subquery_for_set] : prepared_sets->detachSubqueries()) + for (auto & [description, subquery_for_set] : subqueries_for_sets) { if (!subquery_for_set.hasSource()) continue; @@ -166,4 +163,12 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, query_plan.unitePlans(std::move(creating_sets), std::move(plans)); } +void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context) +{ + if (!prepared_sets || prepared_sets->empty()) + return; + + addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(), context); +} + } diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 9c61eb2012c..9995af7bca7 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -49,6 +49,8 @@ private: Processors processors; }; +void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context); + void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context); } From a06832508e459056b62c5fe694143ea006ed83cf Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 7 Sep 2022 10:44:40 +0200 Subject: [PATCH 098/188] Fixed tests --- src/Planner/ActionsChain.h | 2 +- .../02366_explain_query_tree.reference | 49 +++++++++++-------- 2 files changed, 29 insertions(+), 22 deletions(-) diff --git a/src/Planner/ActionsChain.h b/src/Planner/ActionsChain.h index d9c168c1d96..96386a8f309 100644 --- a/src/Planner/ActionsChain.h +++ b/src/Planner/ActionsChain.h @@ -53,7 +53,7 @@ public: /** Initialize actions step with actions dag. * Input column names initialized using actions dag nodes with INPUT type. * - * If available output columns strategy is ALL_NODES, then avaiable output columns initialized using actions dag nodes + * If available output columns strategy is ALL_NODES, then available output columns initialized using actions dag nodes * with INPUT, FUNCTION, ALIAS, ARRAY_JOIN types. * If available output columns strategy is OUTPUT_NODES, then available output columns initialized using actions dag output nodes * with INPUT, FUNCTION, ALIAS, ARRAY_JOIN types. diff --git a/tests/queries/0_stateless/02366_explain_query_tree.reference b/tests/queries/0_stateless/02366_explain_query_tree.reference index 42cc6777e42..d2a0e875621 100644 --- a/tests/queries/0_stateless/02366_explain_query_tree.reference +++ b/tests/queries/0_stateless/02366_explain_query_tree.reference @@ -1,11 +1,11 @@ -QUERY id: 0, is_subquery: 0, is_cte: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 PROJECTION LIST id: 1, nodes: 1 - CONSTANT id: 2, value: UInt64_1, result_type: UInt8 + CONSTANT id: 2, constant_value: UInt64_1, constant_value_type: UInt8 JOIN TREE IDENTIFIER id: 3, identifier: system.one -- -QUERY id: 0, is_subquery: 0, is_cte: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 PROJECTION LIST id: 1, nodes: 2 IDENTIFIER id: 2, identifier: id @@ -13,7 +13,10 @@ QUERY id: 0, is_subquery: 0, is_cte: 0 JOIN TREE IDENTIFIER id: 4, identifier: test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 + PROJECTION COLUMNS + id UInt64 + value String PROJECTION LIST id: 1, nodes: 2 COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3 @@ -21,10 +24,10 @@ QUERY id: 0, is_subquery: 0, is_cte: 0 JOIN TREE TABLE id: 3, table_name: default.test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 PROJECTION LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: arrayMap + FUNCTION id: 2, function_name: arrayMap, is_aggregate_function: 0 ARGUMENTS LIST id: 3, nodes: 2 LAMBDA id: 4 @@ -32,36 +35,38 @@ QUERY id: 0, is_subquery: 0, is_cte: 0 LIST id: 5, nodes: 1 IDENTIFIER id: 6, identifier: x EXPRESSION - FUNCTION id: 7, function_name: plus + FUNCTION id: 7, function_name: plus, is_aggregate_function: 0 ARGUMENTS LIST id: 8, nodes: 2 IDENTIFIER id: 9, identifier: x IDENTIFIER id: 10, identifier: id - CONSTANT id: 11, value: Array_[UInt64_1, UInt64_2, UInt64_3], result_type: Array(UInt8) + CONSTANT id: 11, constant_value: Array_[UInt64_1, UInt64_2, UInt64_3], constant_value_type: Array(UInt8) JOIN TREE IDENTIFIER id: 12, identifier: test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 + PROJECTION COLUMNS + arrayMap(lambda(tuple(x), plus(x, 1)), [1, 2, 3]) Array(UInt16) PROJECTION LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: arrayMap, result_type: Array(UInt16) + FUNCTION id: 2, function_name: arrayMap, is_aggregate_function: 0, result_type: Array(UInt16) ARGUMENTS LIST id: 3, nodes: 2 - LAMBDA id: 4, result_type: UInt16 + LAMBDA id: 4 ARGUMENTS LIST id: 5, nodes: 1 COLUMN id: 6, column_name: x, result_type: UInt8, source_id: 4 EXPRESSION - FUNCTION id: 7, function_name: plus, result_type: UInt16 + FUNCTION id: 7, function_name: plus, is_aggregate_function: 0, result_type: UInt16 ARGUMENTS LIST id: 8, nodes: 2 COLUMN id: 6, column_name: x, result_type: UInt8, source_id: 4 - CONSTANT id: 9, value: UInt64_1, result_type: UInt8 - CONSTANT id: 10, value: Array_[UInt64_1, UInt64_2, UInt64_3], result_type: Array(UInt8) + CONSTANT id: 9, constant_value: UInt64_1, constant_value_type: UInt8 + CONSTANT id: 10, constant_value: Array_[UInt64_1, UInt64_2, UInt64_3], constant_value_type: Array(UInt8) JOIN TREE TABLE id: 11, table_name: default.test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 WITH LIST id: 1, nodes: 1 LAMBDA id: 2, alias: lambda @@ -69,27 +74,29 @@ QUERY id: 0, is_subquery: 0, is_cte: 0 LIST id: 3, nodes: 1 IDENTIFIER id: 4, identifier: x EXPRESSION - FUNCTION id: 5, function_name: plus + FUNCTION id: 5, function_name: plus, is_aggregate_function: 0 ARGUMENTS LIST id: 6, nodes: 2 IDENTIFIER id: 7, identifier: x - CONSTANT id: 8, value: UInt64_1, result_type: UInt8 + CONSTANT id: 8, constant_value: UInt64_1, constant_value_type: UInt8 PROJECTION LIST id: 9, nodes: 1 - FUNCTION id: 10, function_name: lambda + FUNCTION id: 10, function_name: lambda, is_aggregate_function: 0 ARGUMENTS LIST id: 11, nodes: 1 IDENTIFIER id: 12, identifier: id JOIN TREE IDENTIFIER id: 13, identifier: test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 + PROJECTION COLUMNS + lambda(id) UInt64 PROJECTION LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: plus, result_type: UInt64 + FUNCTION id: 2, function_name: plus, is_aggregate_function: 0, result_type: UInt64 ARGUMENTS LIST id: 3, nodes: 2 COLUMN id: 4, column_name: id, result_type: UInt64, source_id: 5 - CONSTANT id: 6, value: UInt64_1, result_type: UInt8 + CONSTANT id: 6, constant_value: UInt64_1, constant_value_type: UInt8 JOIN TREE TABLE id: 5, table_name: default.test_table From 890d9678b9a2d7993fc05a2f78d56b38acf8ff02 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 7 Sep 2022 19:09:37 +0200 Subject: [PATCH 099/188] Fixed tests --- src/Analyzer/QueryAnalysisPass.cpp | 91 ++++++++------- src/Analyzer/QueryAnalysisPass.h | 13 +++ src/Analyzer/QueryTreeBuilder.cpp | 11 +- src/Analyzer/QueryTreeBuilder.h | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- .../InterpreterSelectQueryAnalyzer.cpp | 2 +- src/Planner/ActionsChain.cpp | 16 +-- src/Planner/ActionsChain.h | 6 +- src/Planner/Planner.cpp | 105 ++++++++++-------- src/Storages/SelectQueryInfo.h | 8 ++ .../0_stateless/02381_analyzer_join_final.sql | 3 +- 11 files changed, 145 insertions(+), 114 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 53c1530b761..86ff25b801d 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -510,6 +510,7 @@ struct IdentifierResolveScope QueryTreeNodePtr scope_node; IdentifierResolveScope * parent_scope = nullptr; + /// Identifier lookup to result std::unordered_map identifier_lookup_to_result; /// Lambda argument can be expression like constant, column, or it can be function @@ -560,6 +561,12 @@ struct IdentifierResolveScope /// Subquery depth size_t subquery_depth = 0; + /** Scope join tree node for expression. + * Valid only during analysis construction for single expression. + */ + QueryTreeNodePtr expression_join_tree_node; + + TableExpressionData & getTableExpressionDataOrThrow(QueryTreeNodePtr table_expression_node) { auto it = table_expression_node_to_data.find(table_expression_node); @@ -797,36 +804,43 @@ public: : context(std::move(context_)) {} - void resolve(QueryTreeNodePtr node) + void resolve(QueryTreeNodePtr node, const QueryTreeNodePtr & table_expression) { IdentifierResolveScope scope(node, nullptr /*parent_scope*/); auto node_type = node->getNodeType(); - if (node_type == QueryTreeNodeType::UNION) + if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) { - resolveUnion(node, scope); + if (table_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "For query or union analysis table expression must be empty"); + + if (node_type == QueryTreeNodeType::QUERY) + resolveQuery(node, scope); + else + resolveUnion(node, scope); } - else if (node_type == QueryTreeNodeType::QUERY) + else if (node_type == QueryTreeNodeType::CONSTANT || node_type == QueryTreeNodeType::IDENTIFIER || node_type == QueryTreeNodeType::COLUMN || + node_type == QueryTreeNodeType::FUNCTION || node_type == QueryTreeNodeType::LIST) { - resolveQuery(node, scope); - } - else if (node_type == QueryTreeNodeType::LIST) - { - resolveExpressionNodeList(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - } - else if (node_type == QueryTreeNodeType::FUNCTION) - { - resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - } - else if (node_type == QueryTreeNodeType::LAMBDA) - { - resolveLambda(node, {}, scope); + if (!table_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "For expression analysis table expression must not be empty"); + + scope.expression_join_tree_node = table_expression; + validateTableExpressionModifiers(scope.expression_join_tree_node, scope); + initializeTableExpressionColumns(scope.expression_join_tree_node, scope); + + if (node_type == QueryTreeNodeType::LIST) + resolveExpressionNodeList(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + else + resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } else { throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Node {} with type {} is not supported by query analyzer. Supported nodes are query, list, function, lambda.", + "Node {} with type {} is not supported by query analyzer. Supported nodes are union, query, constant, identifier, column, function, list.", node->formatASTForErrorMessage(), node->getNodeTypeName()); } @@ -843,7 +857,7 @@ private: static void validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope); - static void validateTableExpressionModifiers(QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + static void validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); /// Resolve identifier functions @@ -901,7 +915,7 @@ private: void initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope); - void initializeTableExpressionColumns(QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + void initializeTableExpressionColumns(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); void resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope, QueryExpressionsAliasVisitor & expressions_visitor); @@ -950,7 +964,7 @@ QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunction(const std return {}; const auto & create_function_query = user_defined_function->as(); - auto result_node = buildQueryTree(create_function_query->function_core, context); + auto result_node = buildQueryTree(create_function_query->function_core); if (result_node->getNodeType() != QueryTreeNodeType::LAMBDA) throw Exception(ErrorCodes::LOGICAL_ERROR, "SQL user defined function {} must represent lambda expression. Actual {}", @@ -1091,7 +1105,7 @@ void QueryAnalyzer::validateLimitOffsetExpression(QueryTreeNodePtr & expression_ expression_description); } -void QueryAnalyzer::validateTableExpressionModifiers(QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) +void QueryAnalyzer::validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) { auto * table_node = table_expression_node->as(); auto * table_function_node = table_expression_node->as(); @@ -1175,21 +1189,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con table_name = table_identifier[0]; } - if (database_name.empty()) - database_name = context->getCurrentDatabase(); - - /// TODO: Context resolve storage - - auto & database_catalog = DatabaseCatalog::instance(); - auto database = database_catalog.tryGetDatabase(database_name); - if (!database) - throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exists", database_name); - - auto storage = database->tryGetTable(table_name, context); - - if (!storage) - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exists", table_name); - + StorageID storage_id(database_name, table_name); + storage_id = context->resolveStorageID(storage_id); + auto storage = DatabaseCatalog::instance().getTable(storage_id, context); auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); return std::make_shared(std::move(storage), storage_lock, storage_snapshot); @@ -1893,6 +1895,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTree(const Identifie if (auto resolved_identifier = tryResolveIdentifierFromTableColumns(identifier_lookup, scope)) return resolved_identifier; + if (scope.expression_join_tree_node) + return tryResolveIdentifierFromJoinTreeNode(identifier_lookup, scope.expression_join_tree_node, scope); + auto * query_scope_node = scope.scope_node->as(); if (!query_scope_node || !query_scope_node->getJoinTree()) return {}; @@ -2375,7 +2380,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( throw Exception(ErrorCodes::LOGICAL_ERROR, "Unqualified matcher {} resolve unexpected table expression. In scope {}", matcher_node_typed.formatASTForErrorMessage(), - scope_query_node->formatASTForErrorMessage()); + scope.scope_node->formatASTForErrorMessage()); } for (auto & table_expression_column : table_expression_columns) @@ -3917,7 +3922,7 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod } /// Initialize table expression columns for table expression node -void QueryAnalyzer::initializeTableExpressionColumns(QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) +void QueryAnalyzer::initializeTableExpressionColumns(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) { auto * table_node = table_expression_node->as(); auto * query_node = table_expression_node->as(); @@ -3982,7 +3987,7 @@ void QueryAnalyzer::initializeTableExpressionColumns(QueryTreeNodePtr & table_ex if (column_default && column_default->kind == ColumnDefaultKind::Alias) { - auto column_node = std::make_shared(column_name_and_type, buildQueryTree(column_default->expression, context), table_expression_node); + auto column_node = std::make_shared(column_name_and_type, buildQueryTree(column_default->expression), table_expression_node); column_name_to_column_node.emplace(column_name_and_type.name, column_node); alias_columns_to_resolve.emplace_back(column_name_and_type.name, column_node); } @@ -4829,10 +4834,14 @@ void QueryAnalyzer::resolveUnion(const QueryTreeNodePtr & union_node, Identifier } +QueryAnalysisPass::QueryAnalysisPass(QueryTreeNodePtr table_expression_) + : table_expression(std::move(table_expression_)) +{} + void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { QueryAnalyzer analyzer(std::move(context)); - analyzer.resolve(query_tree_node); + analyzer.resolve(query_tree_node, table_expression); } } diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index 4ae9db8333f..8ebf634e9d8 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -57,6 +57,17 @@ namespace DB class QueryAnalysisPass final : public IQueryTreePass { public: + /** Construct query analysis pass for query or union analysis. + * Available columns are extracted from query node join tree. + */ + QueryAnalysisPass() = default; + + /** Construct query analysis pass for expression or list of expressions analysis. + * Available expression columns are extracted from table expression. + * Table expression node must have query, union, table, table function type. + */ + explicit QueryAnalysisPass(QueryTreeNodePtr table_expression_); + String getName() override { return "QueryAnalysis"; @@ -69,6 +80,8 @@ public: void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; +private: + QueryTreeNodePtr table_expression; }; } diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 49ad3ea805f..168203f3f91 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -64,7 +64,7 @@ namespace class QueryTreeBuilder : public WithContext { public: - QueryTreeBuilder(ASTPtr query_, ContextPtr context_); + explicit QueryTreeBuilder(ASTPtr query_); QueryTreeNodePtr getQueryTreeNode() { @@ -97,9 +97,8 @@ private: }; -QueryTreeBuilder::QueryTreeBuilder(ASTPtr query_, ContextPtr context_) - : WithContext(context_) - , query(query_->clone()) +QueryTreeBuilder::QueryTreeBuilder(ASTPtr query_) + : query(query_->clone()) { if (query->as() || query->as() || @@ -763,9 +762,9 @@ ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & } -QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context) +QueryTreeNodePtr buildQueryTree(ASTPtr query) { - QueryTreeBuilder builder(query, context); + QueryTreeBuilder builder(std::move(query)); return builder.getQueryTreeNode(); } diff --git a/src/Analyzer/QueryTreeBuilder.h b/src/Analyzer/QueryTreeBuilder.h index 3c02527436b..ff5978c1161 100644 --- a/src/Analyzer/QueryTreeBuilder.h +++ b/src/Analyzer/QueryTreeBuilder.h @@ -14,6 +14,6 @@ namespace DB * AST that represent a list of expressions ASTExpressionList. * AST that represent expression ASTIdentifier, ASTAsterisk, ASTLiteral, ASTFunction. */ -QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context); +QueryTreeNodePtr buildQueryTree(ASTPtr query); } diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index f3580902c3b..6fffa1fbfc8 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -377,7 +377,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT is supported for EXPLAIN QUERYTREE query"); auto settings = checkAndGetSettings(ast.getSettings()); - auto query_tree = buildQueryTree(ast.getExplainedQuery(), getContext()); + auto query_tree = buildQueryTree(ast.getExplainedQuery()); if (settings.run_passes) { diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index bafc2cc2644..347903b84c4 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -88,7 +88,7 @@ ASTPtr normalizeAndValidateQuery(const ASTPtr & query) QueryTreeNodePtr buildQueryTreeAndRunPasses(const ASTPtr & query, const ContextPtr & context) { - auto query_tree = buildQueryTree(query, context); + auto query_tree = buildQueryTree(query); QueryTreePassManager query_tree_pass_manager(context); addQueryTreePasses(query_tree_pass_manager); diff --git a/src/Planner/ActionsChain.cpp b/src/Planner/ActionsChain.cpp index 89199628187..d300b688389 100644 --- a/src/Planner/ActionsChain.cpp +++ b/src/Planner/ActionsChain.cpp @@ -72,24 +72,12 @@ void ActionsChainStep::initialize() if (available_output_columns_strategy == AvailableOutputColumnsStrategy::ALL_NODES) { for (const auto & node : actions->getNodes()) - { - if (node.type == ActionsDAG::ActionType::INPUT || - node.type == ActionsDAG::ActionType::FUNCTION || - node.type == ActionsDAG::ActionType::ALIAS || - node.type == ActionsDAG::ActionType::ARRAY_JOIN) - available_output_columns.emplace_back(node.column, node.result_type, node.result_name); - } + available_output_columns.emplace_back(node.column, node.result_type, node.result_name); } else if (available_output_columns_strategy == AvailableOutputColumnsStrategy::OUTPUT_NODES) { for (const auto & node : actions->getOutputs()) - { - if (node->type == ActionsDAG::ActionType::INPUT || - node->type == ActionsDAG::ActionType::FUNCTION || - node->type == ActionsDAG::ActionType::ALIAS || - node->type == ActionsDAG::ActionType::ARRAY_JOIN) - available_output_columns.emplace_back(node->column, node->result_type, node->result_name); - } + available_output_columns.emplace_back(node->column, node->result_type, node->result_name); } available_output_columns.insert(available_output_columns.end(), additional_output_columns.begin(), additional_output_columns.end()); diff --git a/src/Planner/ActionsChain.h b/src/Planner/ActionsChain.h index 96386a8f309..9a4c003c1ea 100644 --- a/src/Planner/ActionsChain.h +++ b/src/Planner/ActionsChain.h @@ -53,10 +53,8 @@ public: /** Initialize actions step with actions dag. * Input column names initialized using actions dag nodes with INPUT type. * - * If available output columns strategy is ALL_NODES, then available output columns initialized using actions dag nodes - * with INPUT, FUNCTION, ALIAS, ARRAY_JOIN types. - * If available output columns strategy is OUTPUT_NODES, then available output columns initialized using actions dag output nodes - * with INPUT, FUNCTION, ALIAS, ARRAY_JOIN types. + * If available output columns strategy is ALL_NODES, then available output columns initialized using actions dag nodes. + * If available output columns strategy is OUTPUT_NODES, then available output columns initialized using actions dag output nodes. */ explicit ActionsChainStep(ActionsDAGPtr actions_, AvailableOutputColumnsStrategy available_output_columns_stategy_ = AvailableOutputColumnsStrategy::ALL_NODES) : actions(std::move(actions_)) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index ca5b0b8d010..e5041c606ef 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -134,6 +134,8 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); auto table_expression_query_info = select_query_info; + table_expression_query_info.table_expression = table_expression; + if (table_node) table_expression_query_info.table_expression_modifiers = table_node->getTableExpressionModifiers(); else @@ -761,6 +763,7 @@ void Planner::buildQueryPlanIfNeeded() SelectQueryInfo select_query_info; select_query_info.original_query = queryNodeToSelectQuery(query_tree); select_query_info.query = select_query_info.original_query; + select_query_info.planner_context = planner_context; StorageLimitsList storage_limits; storage_limits.push_back(buildStorageLimits(*query_context, select_query_options)); @@ -938,17 +941,51 @@ void Planner::buildQueryPlanIfNeeded() having_action_step_index = actions_chain.getLastStepIndex(); } + chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), projection_input, planner_context); + + auto projection_columns = query_node.getProjectionColumns(); + size_t projection_columns_size = projection_columns.size(); + + Names projection_action_names; + NamesWithAliases projection_action_names_with_display_aliases; + projection_action_names_with_display_aliases.reserve(projection_columns_size); + + auto & projection_actions_outputs = projection_actions->getOutputs(); + size_t projection_outputs_size = projection_actions_outputs.size(); + + if (projection_columns_size != projection_outputs_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "QueryTree projection nodes size mismatch. Expected {}. Actual {}", + projection_outputs_size, + projection_columns_size); + + for (size_t i = 0; i < projection_outputs_size; ++i) + { + auto & projection_column = projection_columns[i]; + const auto * projection_node = projection_actions_outputs[i]; + const auto & projection_node_name = projection_node->result_name; + + projection_action_names.push_back(projection_node_name); + projection_action_names_with_display_aliases.push_back({projection_node_name, projection_column.name}); + } + + auto projection_actions_step = std::make_unique(std::move(projection_actions)); + actions_chain.addStep(std::move(projection_actions_step)); + size_t projection_step_index = actions_chain.getLastStepIndex(); + std::optional before_order_by_step_index; if (query_node.hasOrderBy()) { chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & order_by_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - ActionsDAGPtr actions_dag = std::make_shared(order_by_input); - auto & actions_dag_outputs = actions_dag->getOutputs(); - actions_dag_outputs.clear(); + ActionsDAGPtr before_order_by_actions_dag = std::make_shared(order_by_input); + auto & before_order_by_actions_dag_outputs = before_order_by_actions_dag->getOutputs(); + before_order_by_actions_dag_outputs.clear(); - std::unordered_set order_by_actions_dag_outputs_node_names; + std::unordered_set before_order_by_actions_dag_outputs_node_names; /** We add only sort column sort expression in before ORDER BY actions DAG. * WITH fill expressions must be constant nodes. @@ -957,18 +994,19 @@ void Planner::buildQueryPlanIfNeeded() for (auto & sort_column_node : order_by_node_list.getNodes()) { auto & sort_column_node_typed = sort_column_node->as(); - auto expression_dag_nodes = actions_visitor.visit(actions_dag, sort_column_node_typed.getExpression()); + auto expression_dag_nodes = actions_visitor.visit(before_order_by_actions_dag, sort_column_node_typed.getExpression()); + for (auto & action_dag_node : expression_dag_nodes) { - if (order_by_actions_dag_outputs_node_names.contains(action_dag_node->result_name)) + if (before_order_by_actions_dag_outputs_node_names.contains(action_dag_node->result_name)) continue; - actions_dag_outputs.push_back(action_dag_node); - order_by_actions_dag_outputs_node_names.insert(action_dag_node->result_name); + before_order_by_actions_dag_outputs.push_back(action_dag_node); + before_order_by_actions_dag_outputs_node_names.insert(action_dag_node->result_name); } } - auto actions_step_before_order_by = std::make_unique(std::move(actions_dag)); + auto actions_step_before_order_by = std::make_unique(std::move(before_order_by_actions_dag)); actions_chain.addStep(std::move(actions_step_before_order_by)); before_order_by_step_index = actions_chain.getLastStepIndex(); } @@ -992,41 +1030,12 @@ void Planner::buildQueryPlanIfNeeded() } chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), projection_input, planner_context); + const auto & project_names_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + auto project_names_actions = std::make_shared(project_names_input); - const auto & projection_action_dag_nodes = projection_actions->getOutputs(); - size_t projection_action_dag_nodes_size = projection_action_dag_nodes.size(); - - auto projection_columns = query_node.getProjectionColumns(); - size_t projection_columns_size = projection_columns.size(); - - if (projection_columns_size != projection_action_dag_nodes_size) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "QueryTree projection nodes size mismatch. Expected {}. Actual {}", - projection_action_dag_nodes_size, - projection_columns_size); - - Names projection_action_names; - projection_action_names.reserve(projection_columns_size); - - NamesWithAliases projection_action_names_with_display_aliases; - projection_action_names_with_display_aliases.reserve(projection_columns_size); - - for (size_t i = 0; i < projection_columns_size; ++i) - { - auto & projection_column = projection_columns[i]; - const auto * action_dag_node = projection_action_dag_nodes[i]; - const auto & actions_dag_node_name = action_dag_node->result_name; - - projection_action_names.push_back(actions_dag_node_name); - projection_action_names_with_display_aliases.push_back({actions_dag_node_name, projection_column.name}); - } - - projection_actions->project(projection_action_names_with_display_aliases); - - actions_chain.addStep(std::make_unique(std::move(projection_actions))); - size_t projection_action_step_index = actions_chain.getLastStepIndex(); + project_names_actions->project(projection_action_names_with_display_aliases); + actions_chain.addStep(std::make_unique(std::move(project_names_actions))); + size_t project_names_action_step_index = actions_chain.getLastStepIndex(); // std::cout << "Chain dump before finalize" << std::endl; // std::cout << actions_chain.dump() << std::endl; @@ -1192,6 +1201,12 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(having_step)); } + auto & projection_actions_chain_node = actions_chain.at(projection_step_index); + auto expression_step_projection = std::make_unique(query_plan.getCurrentDataStream(), + projection_actions_chain_node->getActions()); + expression_step_projection->setStepDescription("Projection"); + query_plan.addStep(std::move(expression_step_projection)); + if (query_node.isDistinct()) { const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); @@ -1418,8 +1433,8 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(offsets_step)); } - auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), actions_chain[projection_action_step_index]->getActions()); - projection_step->setStepDescription("Projection"); + auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), actions_chain[project_names_action_step_index]->getActions()); + projection_step->setStepDescription("Project names"); query_plan.addStep(std::move(projection_step)); addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index cbee3365754..58fb37b97e8 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -7,7 +7,9 @@ #include #include #include +#include #include +#include #include @@ -178,6 +180,12 @@ struct SelectQueryInfo ASTPtr view_query; /// Optimized VIEW query ASTPtr original_query; /// Unmodified query for projection analysis + /// Planner context + PlannerContextPtr planner_context; + + /// Storage table expression + QueryTreeNodePtr table_expression; + /// Table expression modifiers for storage std::optional table_expression_modifiers; diff --git a/tests/queries/0_stateless/02381_analyzer_join_final.sql b/tests/queries/0_stateless/02381_analyzer_join_final.sql index 0881492edeb..6870b6e5f98 100644 --- a/tests/queries/0_stateless/02381_analyzer_join_final.sql +++ b/tests/queries/0_stateless/02381_analyzer_join_final.sql @@ -27,7 +27,8 @@ INSERT INTO test_table_join_2 VALUES (0, 1), (1, 1); INSERT INTO test_table_join_2 VALUES (1, 2); SELECT t1.id AS t1_id, t2.id AS t2_id, t1.value AS t1_value, t2.value AS t2_value -FROM test_table_join_1 AS t1 FINAL INNER JOIN test_table_join_2 AS t2 FINAL ON t1.id = t2.id; +FROM test_table_join_1 AS t1 FINAL INNER JOIN test_table_join_2 AS t2 FINAL ON t1.id = t2.id +ORDER BY t1_id; DROP TABLE test_table_join_1; DROP TABLE test_table_join_2; From 85ac02c9dbfa5a0fbcba0edb01bd541eb3911424 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 10 Sep 2022 22:06:41 +0200 Subject: [PATCH 100/188] Planner improve action names --- src/Analyzer/QueryAnalysisPass.cpp | 14 +- src/Planner/CollectSets.cpp | 2 +- src/Planner/CollectTableExpressionData.cpp | 33 ++--- src/Planner/Planner.cpp | 32 ++--- src/Planner/PlannerActionsVisitor.cpp | 25 ++-- src/Planner/PlannerActionsVisitor.h | 12 +- src/Planner/PlannerContext.cpp | 126 +++++++++++++----- src/Planner/PlannerContext.h | 72 ++++++++-- src/Planner/PlannerJoins.cpp | 7 +- src/Planner/TableExpressionData.h | 22 +++ .../02337_analyzer_columns_basic.sql | 2 + .../02339_analyzer_matcher_basic.sql | 2 + 12 files changed, 234 insertions(+), 115 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 86ff25b801d..763f3d53b3c 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -88,8 +88,6 @@ namespace ErrorCodes extern const int INCORRECT_RESULT_OF_SCALAR_SUBQUERY; extern const int BAD_ARGUMENTS; extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS; - extern const int UNKNOWN_DATABASE; - extern const int UNKNOWN_TABLE; extern const int TYPE_MISMATCH; extern const int AMBIGUOUS_IDENTIFIER; extern const int INVALID_WITH_FILL_EXPRESSION; @@ -791,12 +789,6 @@ private: using TableExpressionsAliasVisitor = TableExpressionsAliasVisitorMatcher::Visitor; -struct StorageLockAndSnapshot -{ - TableLockHolder lock; - StorageSnapshotPtr snapshot; -}; - class QueryAnalyzer { public: @@ -851,7 +843,7 @@ private: static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path); - QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); + static QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); void evaluateScalarSubquery(QueryTreeNodePtr & query_tree_node, size_t subquery_depth); @@ -2651,8 +2643,8 @@ void QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const Qu auto & lambda_argument_node_typed = lambda_argument_node->as(); const auto & lambda_argument_name = lambda_argument_node_typed.getIdentifier().getFullName(); - bool has_expression_node = data.scope.alias_name_to_expression_node.count(lambda_argument_name) > 0; - bool has_alias_node = data.scope.alias_name_to_lambda_node.count(lambda_argument_name) > 0; + bool has_expression_node = data.scope.alias_name_to_expression_node.contains(lambda_argument_name); + bool has_alias_node = data.scope.alias_name_to_lambda_node.contains(lambda_argument_name); if (has_expression_node || has_alias_node) { diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 4d9de01dc6b..576bf9b861b 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -46,7 +46,7 @@ public: const auto & global_planner_context = planner_context.getGlobalPlannerContext(); const auto & settings = planner_context.getQueryContext()->getSettingsRef(); - String set_key = global_planner_context->getSetKey(in_second_argument); + String set_key = global_planner_context->createSetKey(in_second_argument); auto prepared_set = global_planner_context->getSetOrNull(set_key); if (prepared_set) diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index bb6a0dec3a1..cf7ead918d4 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -50,15 +50,12 @@ public: if (column_node->hasExpression() && column_source_node->getNodeType() == QueryTreeNodeType::JOIN) return; - auto & table_expression_node_to_data = data.planner_context.getTableExpressionNodeToData(); - - auto [it, _] = table_expression_node_to_data.emplace(column_source_node, TableExpressionData()); - auto & table_expression_columns = it->second; + auto & table_expression_data = data.planner_context.getOrCreateTableExpressionData(column_source_node); if (column_node->hasExpression()) { /// Replace ALIAS column with expression - table_expression_columns.addAliasColumnName(column_node->getColumnName()); + table_expression_data.addAliasColumnName(column_node->getColumnName()); node = column_node->getExpression(); visit(node, data); return; @@ -72,19 +69,12 @@ public: "Expected table, table function, query or union column source. Actual {}", column_source_node->formatASTForErrorMessage()); - bool column_already_exists = table_expression_columns.hasColumn(column_node->getColumnName()); + bool column_already_exists = table_expression_data.hasColumn(column_node->getColumnName()); + if (column_already_exists) + return; - if (!column_already_exists) - { - auto column_identifier = data.planner_context.getColumnUniqueIdentifier(column_source_node, column_node->getColumnName()); - data.planner_context.registerColumnNode(node, column_identifier); - table_expression_columns.addColumn(column_node->getColumn(), column_identifier); - } - else - { - auto column_identifier = table_expression_columns.getColumnIdentifierOrThrow(column_node->getColumnName()); - data.planner_context.registerColumnNode(node, column_identifier); - } + auto column_identifier = data.planner_context.getGlobalPlannerContext()->createColumnIdentifier(node); + table_expression_data.addColumn(column_node->getColumn(), column_identifier); } static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) @@ -101,24 +91,23 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContext & { auto & query_node_typed = query_node->as(); auto table_expressions_nodes = extractTableExpressions(query_node_typed.getJoinTree()); - auto & table_expression_node_to_data = planner_context.getTableExpressionNodeToData(); for (auto & table_expression_node : table_expressions_nodes) { - auto [it, _] = table_expression_node_to_data.emplace(table_expression_node, TableExpressionData()); + auto & table_expression_data = planner_context.getOrCreateTableExpressionData(table_expression_node); if (auto * table_node = table_expression_node->as()) { bool storage_is_remote = table_node->getStorage()->isRemote(); - it->second.setIsRemote(storage_is_remote); + table_expression_data.setIsRemote(storage_is_remote); } else if (auto * table_function_node = table_expression_node->as()) { bool storage_is_remote = table_function_node->getStorage()->isRemote(); - it->second.setIsRemote(storage_is_remote); + table_expression_data.setIsRemote(storage_is_remote); } - if (it->second.isRemote()) + if (table_expression_data.isRemote()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Remote storages are not supported"); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index e5041c606ef..d300db74334 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -119,14 +119,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, QueryPlan query_plan; - auto & table_expression_node_to_data = planner_context->getTableExpressionNodeToData(); - auto it = table_expression_node_to_data.find(table_expression); - if (it == table_expression_node_to_data.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Table expression {} is not registered", - table_expression->formatASTForErrorMessage()); - - auto & table_expression_data = it->second; + auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); if (table_node || table_function_node) { @@ -155,7 +148,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, if (read_additional_column) { - auto column_identifier = planner_context->getColumnUniqueIdentifier(table_expression, read_additional_column->name); + const auto & column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(*read_additional_column, table_expression); column_names.push_back(read_additional_column->name); table_expression_data.addColumn(*read_additional_column, column_identifier); } @@ -295,10 +288,11 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, const auto & cast_type = it->second; auto cast_type_name = cast_type->getName(); + Field cast_type_constant_value(cast_type_name); ColumnWithTypeAndName column; - column.name = "__constant_" + cast_type_name; - column.column = DataTypeString().createColumnConst(0, cast_type_name); + column.name = calculateConstantActionNodeName(cast_type_constant_value); + column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); column.type = std::make_shared(); const auto * cast_type_constant_node = &cast_actions_dag->addColumn(std::move(column)); @@ -335,7 +329,7 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, for (auto & output_node : cast_actions_dag->getOutputs()) { - if (output_node->type == ActionsDAG::ActionType::INPUT && output_node->result_name.starts_with("__column")) + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name)) output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); } @@ -454,7 +448,7 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, for (auto & column_from_joined_table : columns_from_joined_table) { - if (column_from_joined_table.name.starts_with("__column")) + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name)) table_join->addJoinedColumn(column_from_joined_table); } @@ -484,7 +478,7 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, for (auto & output : drop_unused_columns_after_join->getOutputs()) { - if (output->result_name.starts_with("__column")) + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output->result_name)) updated_outputs.push_back(output); } @@ -1114,7 +1108,8 @@ void Planner::buildQueryPlanIfNeeded() : static_cast(settings.max_threads); bool storage_has_evenly_distributed_read = false; - auto & table_expression_node_to_data = planner_context->getTableExpressionNodeToData(); + const auto & table_expression_node_to_data = planner_context->getTableExpressionNodeToData(); + if (table_expression_node_to_data.size() == 1) { auto it = table_expression_node_to_data.begin(); @@ -1323,10 +1318,11 @@ void Planner::buildQueryPlanIfNeeded() if (!interpolate_expression->result_type->equals(*expression_to_interpolate->result_type)) { auto cast_type_name = expression_to_interpolate->result_type->getName(); + Field cast_type_constant_value(cast_type_name); ColumnWithTypeAndName column; - column.name = "__constant_" + cast_type_name; - column.column = DataTypeString().createColumnConst(0, cast_type_name); + column.name = calculateConstantActionNodeName(cast_type_name); + column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); column.type = std::make_shared(); const auto * cast_type_constant_node = &interpolate_actions_dag->addColumn(std::move(column)); @@ -1442,7 +1438,7 @@ void Planner::buildQueryPlanIfNeeded() /// Extend lifetime of context, table locks, storages query_plan.addInterpreterContext(planner_context->getQueryContext()); - for (auto & [table_expression, _] : planner_context->getTableExpressionNodeToData()) + for (const auto & [table_expression, _] : planner_context->getTableExpressionNodeToData()) { if (auto * table_node = table_expression->as()) { diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 4d6d7b3f8d1..bfec9238e83 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -10,6 +10,7 @@ #include #include +#include #include #include @@ -24,6 +25,7 @@ #include #include +#include #include namespace DB @@ -39,12 +41,6 @@ namespace ErrorCodes namespace { -String calculateConstantActionNodeName(const Field & constant_literal, const DataTypePtr & constant_type) -{ - auto constant_name = applyVisitor(FieldVisitorToString(), constant_literal); - return "__constant_" + constant_name + "_" + constant_type->getName(); -} - class ActionsScopeNode { public: @@ -353,7 +349,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma auto in_second_argument = function_node.getArguments().getNodes().at(1); const auto & global_planner_context = planner_context->getGlobalPlannerContext(); - auto set_key = global_planner_context->getSetKey(in_second_argument); + auto set_key = global_planner_context->createSetKey(in_second_argument); auto prepared_set = global_planner_context->getSetOrThrow(set_key); auto column_set = ColumnSet::create(1, std::move(prepared_set)); @@ -526,11 +522,11 @@ String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerConte if (isNameOfInFunction(function_node.getFunctionName())) { const auto & in_second_argument_node = function_node.getArguments().getNodes().at(1); - in_function_second_argument_node_name = planner_context.getGlobalPlannerContext()->getSetKey(in_second_argument_node); + in_function_second_argument_node_name = planner_context.getGlobalPlannerContext()->createSetKey(in_second_argument_node); } WriteBufferFromOwnString buffer; - buffer << "__function_" + function_node.getFunctionName(); + buffer << function_node.getFunctionName(); const auto & function_parameters_nodes = function_node.getParameters().getNodes(); @@ -620,4 +616,15 @@ String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerConte return calculateActionNodeName(node, planner_context, empty_map); } +String calculateConstantActionNodeName(const Field & constant_literal, const DataTypePtr & constant_type) +{ + auto constant_name = applyVisitor(FieldVisitorToString(), constant_literal); + return constant_name + "_" + constant_type->getName(); +} + +String calculateConstantActionNodeName(const Field & constant_literal) +{ + return calculateConstantActionNodeName(constant_literal, applyVisitor(FieldToDataType(), constant_literal)); +} + } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index e699220b0b5..2dc34fb32a1 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -21,9 +21,10 @@ using PlannerContextPtr = std::shared_ptr; * into actions dag. * * Preconditions: - * 1. Column name to identifier map in planner context must be already initialized. - * Identifiers in this map are used as action dag node names for column query tree nodes. - * 2. Sets for IN functions are already collected in global context. + * 1. Table expression data for table expression nodes is collected in planner context. + * For column node, that has column table expression source, identifier for column name in table expression data + * is used as action dag node name. + * 2. Sets for IN functions are already collected in planner global context. * * During actions build, there is special handling for following functions: * 1. Aggregate functions are added in actions dag as INPUT nodes. Aggregate functions arguments are not added. @@ -58,5 +59,10 @@ String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerConte */ String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerContext & planner_context); +/// Calculate action node name for constant +String calculateConstantActionNodeName(const Field & constant_literal, const DataTypePtr & constant_type); + +/// Calculate action node name for constant, data type will be derived from constant literal value +String calculateConstantActionNodeName(const Field & constant_literal); } diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 1391184e5f3..f2be620042e 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -1,6 +1,7 @@ #include #include +#include namespace DB { @@ -10,7 +11,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -GlobalPlannerContext::SetKey GlobalPlannerContext::getSetKey(const QueryTreeNodePtr & set_source_node) const +GlobalPlannerContext::SetKey GlobalPlannerContext::createSetKey(const QueryTreeNodePtr & set_source_node) { auto set_source_hash = set_source_node->getTreeHash(); return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); @@ -56,60 +57,113 @@ void GlobalPlannerContext::registerSubqueryNodeForSet(const SetKey & key, Subque set_key_to_subquery_node.emplace(key, std::move(subquery_node_for_set)); } +const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const QueryTreeNodePtr & column_node) +{ + const auto & column_node_typed = column_node->as(); + auto column_source_node = column_node_typed.getColumnSource(); + + return createColumnIdentifier(column_node_typed.getColumn(), column_source_node); +} + +const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const NameAndTypePair & column, const QueryTreeNodePtr & column_source_node) +{ + std::string column_identifier; + + if (column_source_node->hasAlias()) + column_identifier += column_source_node->getAlias(); + else if (const auto * table_source_node = column_source_node->as()) + column_identifier += table_source_node->getStorageID().getFullNameNotQuoted(); + + if (!column_identifier.empty()) + column_identifier += '.'; + + column_identifier += column.name; + column_identifier += '_' + std::to_string(column_identifiers.size()); + + auto [it, inserted] = column_identifiers.emplace(column_identifier); + assert(inserted); + + return *it; +} + +bool GlobalPlannerContext::hasColumnIdentifier(const ColumnIdentifier & column_identifier) +{ + return column_identifiers.contains(column_identifier); +} + PlannerContext::PlannerContext(ContextPtr query_context_, GlobalPlannerContextPtr global_planner_context_) : query_context(std::move(query_context_)) , global_planner_context(std::move(global_planner_context_)) {} -ColumnIdentifier PlannerContext::getColumnUniqueIdentifier(const QueryTreeNodePtr & column_source_node, std::string column_name) +TableExpressionData & PlannerContext::getOrCreateTableExpressionData(const QueryTreeNodePtr & table_expression_node) { - auto column_unique_prefix = "__column_" + std::to_string(column_identifier_counter); - ++column_identifier_counter; - - std::string debug_identifier_suffix; - - if (column_source_node->hasAlias()) - debug_identifier_suffix += column_source_node->getAlias(); - else if (const auto * table_source_node = column_source_node->as()) - debug_identifier_suffix += table_source_node->getStorageID().getFullNameNotQuoted(); - - if (!column_name.empty()) - debug_identifier_suffix += '.' + column_name; - - if (!debug_identifier_suffix.empty()) - column_unique_prefix += '_' + debug_identifier_suffix; - - return column_unique_prefix; + auto [it, _] = table_expression_node_to_data.emplace(table_expression_node, TableExpressionData()); + return it->second; } -void PlannerContext::registerColumnNode(const QueryTreeNodePtr & column_node, const ColumnIdentifier & column_identifier) +const TableExpressionData & PlannerContext::getTableExpressionDataOrThrow(const QueryTreeNodePtr & table_expression_node) const { - assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); - column_node_to_column_identifier.emplace(column_node, column_identifier); + auto table_expression_data_it = table_expression_node_to_data.find(table_expression_node); + if (table_expression_data_it == table_expression_node_to_data.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Table expression {} is not registered in planner context", + table_expression_node->formatASTForErrorMessage()); + + return table_expression_data_it->second; +} + +TableExpressionData & PlannerContext::getTableExpressionDataOrThrow(const QueryTreeNodePtr & table_expression_node) +{ + auto table_expression_data_it = table_expression_node_to_data.find(table_expression_node); + if (table_expression_data_it == table_expression_node_to_data.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Table expression {} is not registered in planner context", + table_expression_node->formatASTForErrorMessage()); + + return table_expression_data_it->second; +} + +const TableExpressionData * PlannerContext::getTableExpressionDataOrNull(const QueryTreeNodePtr & table_expression_node) const +{ + auto table_expression_data_it = table_expression_node_to_data.find(table_expression_node); + if (table_expression_data_it == table_expression_node_to_data.end()) + return nullptr; + + return &table_expression_data_it->second; +} + +TableExpressionData * PlannerContext::getTableExpressionDataOrNull(const QueryTreeNodePtr & table_expression_node) +{ + auto table_expression_data_it = table_expression_node_to_data.find(table_expression_node); + if (table_expression_data_it == table_expression_node_to_data.end()) + return nullptr; + + return &table_expression_data_it->second; } const ColumnIdentifier & PlannerContext::getColumnNodeIdentifierOrThrow(const QueryTreeNodePtr & column_node) const { - assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); - - auto it = column_node_to_column_identifier.find(column_node); - if (it == column_node_to_column_identifier.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Column identifier is not initialized for column {}", - column_node->formatASTForErrorMessage()); - - return it->second; + auto & column_node_typed = column_node->as(); + const auto & column_name = column_node_typed.getColumnName(); + auto column_source = column_node_typed.getColumnSource(); + const auto & table_expression_data = getTableExpressionDataOrThrow(column_source); + return table_expression_data.getColumnIdentifierOrThrow(column_name); } const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const QueryTreeNodePtr & column_node) const { - assert(column_node->getNodeType() == QueryTreeNodeType::COLUMN); - - auto it = column_node_to_column_identifier.find(column_node); - if (it == column_node_to_column_identifier.end()) + auto & column_node_typed = column_node->as(); + const auto & column_name = column_node_typed.getColumnName(); + auto column_source = column_node_typed.getColumnSourceOrNull(); + if (!column_source) return nullptr; - return &it->second; + const auto * table_expression_data = getTableExpressionDataOrNull(column_source); + if (!table_expression_data) + return nullptr; + + return table_expression_data->getColumnIdentifierOrNull(column_name); } } diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 0dd17a4639f..3c2dfd8e863 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -36,8 +36,8 @@ public: using SetKeyToSet = std::unordered_map; using SetKeyToSubqueryNode = std::unordered_map; - /// Get set key for query node - SetKey getSetKey(const QueryTreeNodePtr & set_source_node) const; + /// Create set key for query node + static SetKey createSetKey(const QueryTreeNodePtr & set_source_node); /// Register set for set key void registerSet(const SetKey & key, SetPtr set); @@ -48,7 +48,7 @@ public: /// Get set for key, if no set is registered logical exception is thrown SetPtr getSetOrThrow(const SetKey & key) const; - /** Register subquery node for set + /** Register subquery node for set. * Subquery node for set node must have QUERY or UNION type and set must be initialized. */ void registerSubqueryNodeForSet(const SetKey & key, SubqueryNodeForSet subquery_node_for_set); @@ -58,10 +58,28 @@ public: { return set_key_to_subquery_node; } + + /** Create column identifier for column node. + * + * Result column identifier is added into context. + */ + const ColumnIdentifier & createColumnIdentifier(const QueryTreeNodePtr & column_node); + + /** Create column identifier for column and column source. + * + * Result column identifier is added into context. + */ + const ColumnIdentifier & createColumnIdentifier(const NameAndTypePair & column, const QueryTreeNodePtr & column_source_node); + + /// Check if context has column identifier + bool hasColumnIdentifier(const ColumnIdentifier & column_identifier); + private: SetKeyToSet set_key_to_set; SetKeyToSubqueryNode set_key_to_subquery_node; + + std::unordered_set column_identifiers; }; using GlobalPlannerContextPtr = std::shared_ptr; @@ -69,39 +87,68 @@ using GlobalPlannerContextPtr = std::shared_ptr; class PlannerContext { public: + /// Create planner context with query context and global planner context PlannerContext(ContextPtr query_context_, GlobalPlannerContextPtr global_planner_context_); + /// Get planner context query context const ContextPtr & getQueryContext() const { return query_context; } + /// Get global planner context const GlobalPlannerContextPtr & getGlobalPlannerContext() const { return global_planner_context; } + /// Get global planner context GlobalPlannerContextPtr & getGlobalPlannerContext() { return global_planner_context; } + /// Get or create table expression data for table expression node. + TableExpressionData & getOrCreateTableExpressionData(const QueryTreeNodePtr & table_expression_node); + + /** Get table expression data. + * Exception is thrown if there are no table expression data for table expression node. + */ + const TableExpressionData & getTableExpressionDataOrThrow(const QueryTreeNodePtr & table_expression_node) const; + + /** Get table expression data. + * Exception is thrown if there are no table expression data for table expression node. + */ + TableExpressionData & getTableExpressionDataOrThrow(const QueryTreeNodePtr & table_expression_node); + + /** Get table expression data. + * Null is returned if there are no table expression data for table expression node. + */ + const TableExpressionData * getTableExpressionDataOrNull(const QueryTreeNodePtr & table_expression_node) const; + + /** Get table expression data. + * Null is returned if there are no table expression data for table expression node. + */ + TableExpressionData * getTableExpressionDataOrNull(const QueryTreeNodePtr & table_expression_node); + + /// Get table expression node to data map read only map const std::unordered_map & getTableExpressionNodeToData() const { return table_expression_node_to_data; } - std::unordered_map & getTableExpressionNodeToData() - { - return table_expression_node_to_data; - } - - ColumnIdentifier getColumnUniqueIdentifier(const QueryTreeNodePtr & column_source_node, std::string column_name = {}); - - void registerColumnNode(const QueryTreeNodePtr & column_node, const ColumnIdentifier & column_identifier); - + /** Get column node identifier. + * For column node source check if table expression data is registered. + * If table expression data is not registered exception is thrown. + * In table expression data get column node identifier using column name. + */ const ColumnIdentifier & getColumnNodeIdentifierOrThrow(const QueryTreeNodePtr & column_node) const; + /** Get column node identifier. + * For column node source check if table expression data is registered. + * If table expression data is not registered null is returned. + * In table expression data get column node identifier or null using column name. + */ const ColumnIdentifier * getColumnNodeIdentifierOrNull(const QueryTreeNodePtr & column_node) const; private: @@ -117,7 +164,6 @@ private: /// Table expression node to data std::unordered_map table_expression_node_to_data; - size_t column_identifier_counter = 0; }; using PlannerContextPtr = std::shared_ptr; diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 57b4792d9bc..e05c83f5734 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -388,9 +388,12 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & right_key_node->result_type->getName()); } + auto cast_type_name = common_type->getName(); + Field cast_type_constant_value(cast_type_name); + ColumnWithTypeAndName cast_column; - cast_column.name = "__constant_" + Field(common_type->getName()).dump(); - cast_column.column = DataTypeString().createColumnConst(0, common_type->getName()); + cast_column.name = calculateConstantActionNodeName(cast_type_constant_value); + cast_column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); cast_column.type = std::make_shared(); const ActionsDAG::Node * cast_type_constant_node = nullptr; diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h index 2708279e1dd..438a3ead114 100644 --- a/src/Planner/TableExpressionData.h +++ b/src/Planner/TableExpressionData.h @@ -13,16 +13,24 @@ namespace ErrorCodes using ColumnIdentifier = std::string; +/** Table expression data is created for each table expression that take part in query. + * Table expression data has information about columns that participate in query, their name to identifier mapping, + * and additional table expression properties. + */ class TableExpressionData { public: using ColumnNameToColumnIdentifier = std::unordered_map; + /// Return true if column with name exists, false otherwise bool hasColumn(const std::string & column_name) const { return alias_columns_names.contains(column_name) || columns_names.contains(column_name); } + /** Add column in table expression data. + * Column identifier must be created using global planner context. + */ void addColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) { if (hasColumn(column.name)) @@ -33,6 +41,9 @@ public: column_name_to_column_identifier.emplace(column.name, column_identifier); } + /** Add column if it does not exists in table expression data. + * Column identifier must be created using global planner context. + */ void addColumnIfNotExists(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) { if (hasColumn(column.name)) @@ -43,31 +54,39 @@ public: column_name_to_column_identifier.emplace(column.name, column_identifier); } + /// Add alias column name void addAliasColumnName(const std::string & column_name) { alias_columns_names.insert(column_name); } + /// Get alias column names const NameSet & getAliasColumnsNames() const { return alias_columns_names; } + /// Get column names const NameSet & getColumnsNames() const { return columns_names; } + /// Get columns const NamesAndTypesList & getColumns() const { return columns; } + /// Get column name to identifier map const ColumnNameToColumnIdentifier & getColumnNameToIdentifier() const { return column_name_to_column_identifier; } + /** Get column identifier for column name. + * Exception is thrown if there are no identifier for column name. + */ const ColumnIdentifier & getColumnIdentifierOrThrow(const std::string & column_name) const { auto it = column_name_to_column_identifier.find(column_name); @@ -79,6 +98,9 @@ public: return it->second; } + /** Get column identifier for column name. + * Null is returned if there are no identifier for column name. + */ const ColumnIdentifier * getColumnIdentifierOrNull(const std::string & column_name) const { auto it = column_name_to_column_identifier.find(column_name); diff --git a/tests/queries/0_stateless/02337_analyzer_columns_basic.sql b/tests/queries/0_stateless/02337_analyzer_columns_basic.sql index e03a65a4e4f..00dc6c1351e 100644 --- a/tests/queries/0_stateless/02337_analyzer_columns_basic.sql +++ b/tests/queries/0_stateless/02337_analyzer_columns_basic.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + SET use_analyzer = 1; -- Empty from section diff --git a/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql b/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql index 5d479293084..862cb2f88ff 100644 --- a/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql +++ b/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + SET use_analyzer = 1; SELECT 'Matchers without FROM section'; From 8cadb1b3186288a3a0bdc9b33c5b692a8b264a86 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 12 Sep 2022 16:14:40 +0200 Subject: [PATCH 101/188] Added WINDOW functions support --- ....cpp => CollectAggregateFunctionNodes.cpp} | 7 +- ...itor.h => CollectAggregateFunctionNodes.h} | 2 +- src/Analyzer/CollectWindowFunctionNodes.cpp | 87 +++ src/Analyzer/CollectWindowFunctionNodes.h | 23 + src/Analyzer/FunctionNode.cpp | 39 +- src/Analyzer/FunctionNode.h | 47 +- src/Analyzer/IQueryTreeNode.cpp | 1 + src/Analyzer/IQueryTreeNode.h | 1 + src/Analyzer/QueryAnalysisPass.cpp | 540 +++++++++++++----- src/Analyzer/QueryAnalysisPass.h | 1 + src/Analyzer/QueryNode.cpp | 29 +- src/Analyzer/QueryNode.h | 40 +- src/Analyzer/QueryTreeBuilder.cpp | 73 +++ src/Analyzer/WindowNode.cpp | 207 +++++++ src/Analyzer/WindowNode.h | 193 +++++++ src/Interpreters/WindowDescription.cpp | 3 +- src/Interpreters/WindowDescription.h | 1 - src/Planner/ActionsChain.cpp | 17 + src/Planner/ActionsChain.h | 4 +- src/Planner/Planner.cpp | 149 ++++- src/Planner/PlannerActionsVisitor.cpp | 126 +++- src/Planner/PlannerActionsVisitor.h | 12 +- src/Planner/PlannerAggregation.cpp | 26 +- src/Planner/PlannerAggregation.h | 3 - src/Planner/PlannerSorting.h | 2 +- src/Planner/PlannerWindowFunctions.cpp | 145 +++++ src/Planner/PlannerWindowFunctions.h | 20 + src/Planner/Utils.cpp | 15 + src/Planner/Utils.h | 4 + ...reuseStorageOrderingForWindowFunctions.cpp | 2 +- 30 files changed, 1619 insertions(+), 200 deletions(-) rename src/Analyzer/{CollectAggregateFunctionVisitor.cpp => CollectAggregateFunctionNodes.cpp} (93%) rename src/Analyzer/{CollectAggregateFunctionVisitor.h => CollectAggregateFunctionNodes.h} (89%) create mode 100644 src/Analyzer/CollectWindowFunctionNodes.cpp create mode 100644 src/Analyzer/CollectWindowFunctionNodes.h create mode 100644 src/Analyzer/WindowNode.cpp create mode 100644 src/Analyzer/WindowNode.h create mode 100644 src/Planner/PlannerWindowFunctions.cpp create mode 100644 src/Planner/PlannerWindowFunctions.h diff --git a/src/Analyzer/CollectAggregateFunctionVisitor.cpp b/src/Analyzer/CollectAggregateFunctionNodes.cpp similarity index 93% rename from src/Analyzer/CollectAggregateFunctionVisitor.cpp rename to src/Analyzer/CollectAggregateFunctionNodes.cpp index 40c2f24cbc0..5a60d63ca8b 100644 --- a/src/Analyzer/CollectAggregateFunctionVisitor.cpp +++ b/src/Analyzer/CollectAggregateFunctionNodes.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -30,10 +30,7 @@ public: static void visit(const QueryTreeNodePtr & node, Data & data) { auto * function_node = node->as(); - if (!function_node) - return; - - if (!function_node->isAggregateFunction()) + if (!function_node || !function_node->isAggregateFunction()) return; if (!data.assert_no_aggregates_place_message.empty()) diff --git a/src/Analyzer/CollectAggregateFunctionVisitor.h b/src/Analyzer/CollectAggregateFunctionNodes.h similarity index 89% rename from src/Analyzer/CollectAggregateFunctionVisitor.h rename to src/Analyzer/CollectAggregateFunctionNodes.h index df26a8f4fc3..5e68ef67d8b 100644 --- a/src/Analyzer/CollectAggregateFunctionVisitor.h +++ b/src/Analyzer/CollectAggregateFunctionNodes.h @@ -15,7 +15,7 @@ QueryTreeNodes collectAggregateFunctionNodes(const QueryTreeNodePtr & node); */ void collectAggregateFunctionNodes(const QueryTreeNodePtr & node, QueryTreeNodes & result); -/** Assert that there are not aggregate function nodes in node children. +/** Assert that there are no aggregate function nodes in node children. * Do not visit subqueries. */ void assertNoAggregateFunctionNodes(const QueryTreeNodePtr & node, const String & assert_no_aggregates_place_message); diff --git a/src/Analyzer/CollectWindowFunctionNodes.cpp b/src/Analyzer/CollectWindowFunctionNodes.cpp new file mode 100644 index 00000000000..25c159c8386 --- /dev/null +++ b/src/Analyzer/CollectWindowFunctionNodes.cpp @@ -0,0 +1,87 @@ +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_AGGREGATION; +} + +namespace +{ + +class CollecWindowFunctionNodesMatcher +{ +public: + using Visitor = ConstInDepthQueryTreeVisitor; + + struct Data + { + Data() = default; + + String assert_no_window_functions_place_message; + QueryTreeNodes * window_function_nodes = nullptr; + }; + + static void visit(const QueryTreeNodePtr & node, Data & data) + { + auto * function_node = node->as(); + if (!function_node || !function_node->isWindowFunction()) + return; + + if (!data.assert_no_window_functions_place_message.empty()) + throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, + "Window function {} is found {} in query", + function_node->getName(), + data.assert_no_window_functions_place_message); + + if (data.window_function_nodes) + data.window_function_nodes->push_back(node); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) + { + return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); + } +}; + +using CollectWindowFunctionNodesVisitor = CollecWindowFunctionNodesMatcher::Visitor; + +} + +QueryTreeNodes collectWindowFunctionNodes(const QueryTreeNodePtr & node) +{ + QueryTreeNodes window_function_nodes; + + CollectWindowFunctionNodesVisitor::Data data; + data.window_function_nodes = &window_function_nodes; + + CollectWindowFunctionNodesVisitor visitor(data); + visitor.visit(node); + + return window_function_nodes; +} + +void collectWindowFunctionNodes(const QueryTreeNodePtr & node, QueryTreeNodes & result) +{ + CollectWindowFunctionNodesVisitor::Data data; + data.window_function_nodes = &result; + + CollectWindowFunctionNodesVisitor visitor(data); + visitor.visit(node); +} + +void assertNoWindowFunctionNodes(const QueryTreeNodePtr & node, const String & assert_no_window_functions_place_message) +{ + CollectWindowFunctionNodesVisitor::Data data; + data.assert_no_window_functions_place_message = assert_no_window_functions_place_message; + + CollectWindowFunctionNodesVisitor(data).visit(node); +} + +} diff --git a/src/Analyzer/CollectWindowFunctionNodes.h b/src/Analyzer/CollectWindowFunctionNodes.h new file mode 100644 index 00000000000..b6ff5f22f93 --- /dev/null +++ b/src/Analyzer/CollectWindowFunctionNodes.h @@ -0,0 +1,23 @@ +#pragma once + +#include + +namespace DB +{ + +/** Collect window function nodes in node children. + * Do not visit subqueries. + */ +QueryTreeNodes collectWindowFunctionNodes(const QueryTreeNodePtr & node); + +/** Collect window function nodes in node children and add them into result. + * Do not visit subqueries. + */ +void collectWindowFunctionNodes(const QueryTreeNodePtr & node, QueryTreeNodes & result); + +/** Assert that there are no window function nodes in node children. + * Do not visit subqueries. + */ +void assertNoWindowFunctionNodes(const QueryTreeNodePtr & node, const String & assert_no_window_functions_place_message); + +} diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index 2e31d2da6c8..e9d1534cb85 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -6,10 +6,13 @@ #include #include +#include + #include + #include -#include +#include namespace DB { @@ -17,7 +20,7 @@ namespace DB FunctionNode::FunctionNode(String function_name_) : function_name(function_name_) { - children.resize(2); + children.resize(children_size); children[parameters_child_index] = std::make_shared(); children[arguments_child_index] = std::make_shared(); } @@ -38,6 +41,11 @@ void FunctionNode::resolveAsAggregateFunction(AggregateFunctionPtr aggregate_fun function_name = aggregate_function->getName(); } +void FunctionNode::resolveAsWindowFunction(AggregateFunctionPtr window_function_value, DataTypePtr result_type_value) +{ + resolveAsAggregateFunction(window_function_value, result_type_value); +} + void FunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "FUNCTION id: " << format_state.getNodeId(this); @@ -46,7 +54,14 @@ void FunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state buffer << ", alias: " << getAlias(); buffer << ", function_name: " << function_name; - buffer << ", is_aggregate_function: " << isAggregateFunction(); + + std::string function_type = "ordinary"; + if (isAggregateFunction()) + function_type = "aggregate"; + else if (isWindowFunction()) + function_type = "window"; + + buffer << ", function_type: " << function_type; if (result_type) buffer << ", result_type: " + result_type->getName(); @@ -70,6 +85,12 @@ void FunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state buffer << '\n' << std::string(indent + 2, ' ') << "ARGUMENTS\n"; arguments.dumpTreeImpl(buffer, format_state, indent + 4); } + + if (hasWindow()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "WINDOW\n"; + getWindowNode()->dumpTreeImpl(buffer, format_state, indent + 4); + } } String FunctionNode::getName() const @@ -123,6 +144,7 @@ void FunctionNode::updateTreeHashImpl(HashState & hash_state) const hash_state.update(function_name.size()); hash_state.update(function_name); hash_state.update(isAggregateFunction()); + hash_state.update(isWindowFunction()); if (result_type) { @@ -148,6 +170,16 @@ ASTPtr FunctionNode::toASTImpl() const auto function_ast = std::make_shared(); function_ast->name = function_name; + function_ast->is_window_function = isWindowFunction(); + + auto window_node = getWindowNode(); + if (window_node) + { + if (auto * identifier_node = window_node->as()) + function_ast->window_name = identifier_node->getIdentifier().getFullName(); + else + function_ast->window_definition = window_node->toAST(); + } const auto & parameters = getParameters(); if (!parameters.getNodes().empty()) @@ -166,6 +198,7 @@ ASTPtr FunctionNode::toASTImpl() const QueryTreeNodePtr FunctionNode::cloneImpl() const { auto result_function = std::make_shared(function_name); + /// This is valid for clone method function or aggregate function must be stateless result_function->function = function; result_function->aggregate_function = aggregate_function; diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index 66339d91890..431c896473c 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -23,9 +23,12 @@ using AggregateFunctionPtr = std::shared_ptr; * Function can be: * 1. Aggregate function. Example: quantile(0.5)(x), sum(x). * 2. Non aggregate function. Example: plus(x, x). + * 3. Window function. Example: sum(x) OVER (PARTITION BY expr ORDER BY expr). * * Initially function node is initialize with function name. - * During query analysis pass function must be resolved using `resolveAsFunction` or `resolveAsAggregateFunction` methods. + * For window function client must initialize function window node. + * + * During query analysis pass function must be resolved using `resolveAsFunction`, `resolveAsAggregateFunction`, `resolveAsWindowFunction` methods. * Resolved function is function that has result type and is initialized with concrete aggregate or non aggregate function. */ class FunctionNode; @@ -35,7 +38,7 @@ class FunctionNode final : public IQueryTreeNode { public: /** Initialize function node with function name. - * Later during query analysis path function must be resolved. + * Later during query analysis pass function must be resolved. */ explicit FunctionNode(String function_name_); @@ -93,6 +96,30 @@ public: return children[arguments_child_index]; } + /// Has window + bool hasWindow() const + { + return children[window_child_index] != nullptr; + } + + /** Get window node. + * Valid only for window function node. + * Can be identifier if window function is defined as expr OVER window_name. + * Or can be window node if window function is defined as expr OVER (window_name ...). + */ + const QueryTreeNodePtr & getWindowNode() const + { + return children[window_child_index]; + } + + /** Get window node. + * Valid only for window function node. + */ + QueryTreeNodePtr & getWindowNode() + { + return children[window_child_index]; + } + /** Get non aggregate function. * If function is not resolved nullptr returned. */ @@ -116,10 +143,16 @@ public: return result_type != nullptr && (function != nullptr || aggregate_function != nullptr); } + /// Is window function + bool isWindowFunction() const + { + return getWindowNode() != nullptr; + } + /// Is function node resolved as aggregate function bool isAggregateFunction() const { - return aggregate_function != nullptr; + return aggregate_function != nullptr && !isWindowFunction(); } /// Is function node resolved as non aggregate function @@ -142,6 +175,12 @@ public: */ void resolveAsAggregateFunction(AggregateFunctionPtr aggregate_function_value, DataTypePtr result_type_value); + /** Resolve function node as window function. + * It is important that function name is update with resolved function name. + * Main motivation for this is query tree optimizations. + */ + void resolveAsWindowFunction(AggregateFunctionPtr window_function_value, DataTypePtr result_type_value); + /// Perform constant folding for function node void performConstantFolding(ConstantValuePtr constant_folded_value) { @@ -179,6 +218,8 @@ protected: private: static constexpr size_t parameters_child_index = 0; static constexpr size_t arguments_child_index = 1; + static constexpr size_t window_child_index = 2; + static constexpr size_t children_size = window_child_index + 1; String function_name; FunctionOverloadResolverPtr function; diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 7a8023b05e2..71ff344deb4 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -32,6 +32,7 @@ const char * toString(QueryTreeNodeType type) case QueryTreeNodeType::LAMBDA: return "LAMBDA"; case QueryTreeNodeType::SORT_COLUMN: return "SORT_COLUMN"; case QueryTreeNodeType::INTERPOLATE_COLUMN: return "INTERPOLATE_COLUMN"; + case QueryTreeNodeType::WINDOW: return "WINDOW"; case QueryTreeNodeType::TABLE: return "TABLE"; case QueryTreeNodeType::TABLE_FUNCTION: return "TABLE_FUNCTION"; case QueryTreeNodeType::QUERY: return "QUERY"; diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 58762411025..03de250f403 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -39,6 +39,7 @@ enum class QueryTreeNodeType LAMBDA, SORT_COLUMN, INTERPOLATE_COLUMN, + WINDOW, TABLE, TABLE_FUNCTION, QUERY, diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 763f3d53b3c..12d1d069611 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -42,6 +42,7 @@ #include #include #include +#include #include #include #include @@ -50,7 +51,8 @@ #include #include #include -#include +#include +#include #include @@ -94,6 +96,7 @@ namespace ErrorCodes extern const int INVALID_LIMIT_EXPRESSION; extern const int EMPTY_LIST_OF_COLUMNS_QUERIED; extern const int TOO_DEEP_SUBQUERIES; + extern const int UNKNOWN_AGGREGATE_FUNCTION; extern const int NOT_AN_AGGREGATE; extern const int ILLEGAL_AGGREGATION; extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; @@ -190,7 +193,6 @@ namespace ErrorCodes * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. * TODO: JOIN support SELF JOIN with MergeTree. JOIN support matchers. - * TODO: WINDOW functions * TODO: Support function identifier resolve from parent query scope, if lambda in parent scope does not capture any columns. * TODO: Support group_by_use_nulls */ @@ -529,6 +531,9 @@ struct IdentifierResolveScope /// CTE name to query node std::unordered_map cte_name_to_query_node; + /// Window name to window node + std::unordered_map window_name_to_window_node; + /// Nodes with duplicated aliases std::unordered_set nodes_with_duplicated_aliases; @@ -564,6 +569,33 @@ struct IdentifierResolveScope */ QueryTreeNodePtr expression_join_tree_node; + [[maybe_unused]] const IdentifierResolveScope * getNearestQueryScope() const + { + const IdentifierResolveScope * scope_to_check = this; + while (scope_to_check != nullptr) + { + if (scope_to_check->scope_node->getNodeType() == QueryTreeNodeType::QUERY) + break; + + scope_to_check = scope_to_check->parent_scope; + } + + return scope_to_check; + } + + IdentifierResolveScope * getNearestQueryScope() + { + IdentifierResolveScope * scope_to_check = this; + while (scope_to_check != nullptr) + { + if (scope_to_check->scope_node->getNodeType() == QueryTreeNodeType::QUERY) + break; + + scope_to_check = scope_to_check->parent_scope; + } + + return scope_to_check; + } TableExpressionData & getTableExpressionDataOrThrow(QueryTreeNodePtr table_expression_node) { @@ -851,6 +883,8 @@ private: static void validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope); + /// Resolve identifier functions QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier); @@ -889,6 +923,8 @@ private: QueryTreeNodePtr resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); + void resolveWindow(QueryTreeNodePtr & window_node, IdentifierResolveScope & scope); + void resolveLambda(const QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope); void resolveFunction(QueryTreeNodePtr & function_node, IdentifierResolveScope & scope); @@ -901,6 +937,8 @@ private: void resolveInterpolateColumnsNodeList(QueryTreeNodePtr & sort_columns_node_list, IdentifierResolveScope & scope); + void resolveWindowNodeList(QueryTreeNodePtr & window_node_list, IdentifierResolveScope & scope); + String calculateProjectionNodeDisplayName(QueryTreeNodePtr & node, IdentifierResolveScope & scope); NamesAndTypes resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope); @@ -1157,6 +1195,52 @@ void QueryAnalyzer::validateTableExpressionModifiers(const QueryTreeNodePtr & ta } } +void QueryAnalyzer::mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope) +{ + auto & window_node_typed = window_node->as(); + auto parent_window_name = window_node_typed.getParentWindowName(); + + auto & parent_window_node_typed = parent_window_node->as(); + + // If an existing_window_name is specified it must refer to an earlier + // entry in the WINDOW list; the new window copies its partitioning clause + // from that entry, as well as its ordering clause if any. In this case + // the new window cannot specify its own PARTITION BY clause, and it can + // specify ORDER BY only if the copied window does not have one. The new + // window always uses its own frame clause; the copied window must not + // specify a frame clause. + // -- https://www.postgresql.org/docs/current/sql-select.html + if (window_node_typed.hasPartitionBy()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Derived window definition '{}' is not allowed to override PARTITION BY. In scope {}", + window_node_typed.formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + + if (window_node_typed.hasOrderBy() && parent_window_node_typed.hasOrderBy()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Derived window definition '{}' is not allowed to override a non-empty ORDER BY. In scope {}", + window_node_typed.formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + + if (!parent_window_node_typed.getWindowFrame().is_default) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Parent window '{}' is not allowed to define a frame: while processing derived window definition '{}'. In scope {}", + parent_window_name, + window_node_typed.formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + + window_node_typed.getPartitionByNode() = parent_window_node_typed.getPartitionBy().clone(); + + if (parent_window_node_typed.hasOrderBy()) + window_node_typed.getOrderByNode() = parent_window_node_typed.getOrderBy().clone(); +} + /// Resolve identifier functions implementation /// Try resolve table identifier from database catalog @@ -2581,6 +2665,92 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, return list; } +/** Resolve window function window node. + * + * Node can be identifier or window node. + * Example: SELECT count(*) OVER w FROM test_table WINDOW w AS (PARTITION BY id); + * Example: SELECT count(*) OVER (PARTITION BY id); + * + * If node has parent window name specified, then parent window definition is searched in nearest query scope WINDOW section. + * If node is identifier, than node is replaced with window definition. + * If node is window, that window node is merged with parent window node. + * + * Window node PARTITION BY and ORDER BY parts are resolved. + * If window node has frame begin OFFSET or frame end OFFSET specified, they are resolved, and window node frame constants are updated. + * Window node frame is validated. + */ +void QueryAnalyzer::resolveWindow(QueryTreeNodePtr & node, IdentifierResolveScope & scope) +{ + std::string parent_window_name; + auto * identifier_node = node->as(); + + if (identifier_node) + { + parent_window_name = identifier_node->getIdentifier().getFullName(); + } + else if (auto * window_node = node->as()) + { + parent_window_name = window_node->getParentWindowName(); + } + + if (!parent_window_name.empty()) + { + auto * nearest_query_scope = scope.getNearestQueryScope(); + + if (!nearest_query_scope) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Window '{}' does not exists.", parent_window_name); + + auto & scope_window_name_to_window_node = nearest_query_scope->window_name_to_window_node; + + auto window_node_it = scope_window_name_to_window_node.find(parent_window_name); + if (window_node_it == scope_window_name_to_window_node.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Window '{}' does not exists. In scope {}", + parent_window_name, + nearest_query_scope->scope_node->formatASTForErrorMessage()); + + if (identifier_node) + node = window_node_it->second->clone(); + else + mergeWindowWithParentWindow(node, window_node_it->second, scope); + } + + auto & window_node = node->as(); + + window_node.setParentWindowName({}); + resolveExpressionNodeList(window_node.getPartitionByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + resolveSortColumnsNodeList(window_node.getOrderByNode(), scope); + + if (window_node.hasFrameBeginOffset()) + { + resolveExpressionNode(window_node.getFrameBeginOffsetNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + const auto window_frame_begin_constant_value = window_node.getFrameBeginOffsetNode()->getConstantValueOrNull(); + if (!window_frame_begin_constant_value || !isNativeNumber(removeNullable(window_frame_begin_constant_value->getType()))) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Window frame begin OFFSET expression must be constant with numeric type. Actual {}. In scope {}", + window_node.getFrameBeginOffsetNode()->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + window_node.getWindowFrame().begin_offset = window_frame_begin_constant_value->getValue(); + } + + if (window_node.hasFrameEndOffset()) + { + resolveExpressionNode(window_node.getFrameEndOffsetNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + const auto window_frame_end_constant_value = window_node.getFrameEndOffsetNode()->getConstantValueOrNull(); + if (!window_frame_end_constant_value || !isNativeNumber(removeNullable(window_frame_end_constant_value->getType()))) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Window frame begin OFFSET expression must be constant with numeric type. Actual {}. In scope {}", + window_node.getFrameEndOffsetNode()->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + window_node.getWindowFrame().end_offset = window_frame_end_constant_value->getValue(); + } + + window_node.getWindowFrame().checkValid(); +} /** Resolve lambda function. * This function modified lambda_node during resolve. It is caller responsibility to clone lambda before resolve @@ -2673,25 +2843,24 @@ void QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const Qu * * Steps: * 1. Resolve function parameters. Validate that each function parameter must be constant node. - * 2. Resolve function arguments list, lambda expressions are allowed as function arguments. - * 3. Initialize argument_columns, argument_types, function_lambda_arguments_indexes arrays from function arguments. - * 4. Try to resolve function node name identifier as function. - * 5. If function name identifier was not resolved as function, try to lookup lambda from sql user defined functions factory. - * 6. If function was resolve as lambda from step 4, or 5, then resolve lambda using function arguments and replace function node with lambda result. + * 2. Try to lookup function as lambda in current scope. If it is lambda we can skip `in` and `count` special handling. + * 3. If function is count function, that take unqualified ASTERISK matcher, remove it from its arguments. Example: SELECT count(*) FROM test_table; + * 4. If function is `IN` function, then right part of `IN` function is replaced as subquery. + * 5. Resolve function arguments list, lambda expressions are allowed as function arguments. + * For `IN` function table expressions are allowed as function arguments. + * 6. Initialize argument_columns, argument_types, function_lambda_arguments_indexes arrays from function arguments. + * 7. If function name identifier was not resolved as function in current scope, try to lookup lambda from sql user defined functions factory. + * 8. If function was resolve as lambda from step 2 or 7, then resolve lambda using function arguments and replace function node with lambda result. * After than function node is resolved. - * 7. If function was not resolved during step 6 as lambda, then try to resolve function as executable user defined function or aggregate function or - * non aggregate function. + * 9. If function was not resolved during step 6 as lambda, then try to resolve function as window function or executable user defined function + * or ordinary function or aggregate function. * - * Special case is `untuple` function that takes single compound argument expression. If argument is not compound expression throw exception. - * Wrap compound expression subcolumns into `tupleElement` and replace function node with them. After that `untuple` function node is resolved. - * - * If function is resolved as executable user defined function or aggregate function, function node is resolved + * If function is resolved as window function or executable user defined function or aggregate function, function node is resolved * no additional special handling is required. * - * 8. If function was resolved as non aggregate function. Then if on step 3 there were lambdas, their result types need to be initialized and + * 8. If function was resolved as non aggregate function. Then if some of function arguments are lambda expressions, their result types need to be initialized and * they must be resolved. - * 9. If function is suitable for constant folding, try to replace function node with constant result. - * + * 9. If function is suitable for constant folding, try to perform constant folding for function node. */ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveScope & scope) { @@ -2726,7 +2895,23 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc parameters.push_back(constant_value->getValue()); } - bool is_special_function_in = isNameOfInFunction(function_name); + //// If function node is not window function try to lookup function node name as lambda identifier. + QueryTreeNodePtr lambda_expression_untyped; + if (!function_node.isWindowFunction()) + { + auto function_lookup_result = tryResolveIdentifier({Identifier{function_node.getFunctionName()}, IdentifierLookupContext::FUNCTION}, scope); + lambda_expression_untyped = function_lookup_result.resolved_identifier; + } + + bool is_special_function_in = false; + if (!lambda_expression_untyped) + { + is_special_function_in = isNameOfInFunction(function_name); + + /// Handle SELECT count(*) FROM test_table + if (function_name == "count") + function_node.getArguments().getNodes().clear(); + } /// Resolve function arguments @@ -2844,105 +3029,125 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc argument_columns.emplace_back(std::move(argument_column)); } - /** Lookup function node name as lambda identifier. - * If no lambda node exists with function node name identifier, try to resolve it as lambda from sql user defined functions. + /** Try to resolve function as + * 1. Lambda function in current scope. Example: WITH (x -> x + 1) AS lambda SELECT lambda(1); + * 2. Lambda function from sql user defined functions. + * 3. Special `untuple` function. + * 4. Special `grouping` function. + * 5. Window function. + * 6. Executable user defined function. + * 7. Ordinary function. + * 8. Aggregate function. + * + * TODO: Provide better error hints. */ - auto function_lookup_result = tryResolveIdentifier({Identifier{function_node.getFunctionName()}, IdentifierLookupContext::FUNCTION}, scope); - auto lambda_expression_untyped = function_lookup_result.resolved_identifier; - if (!lambda_expression_untyped) - lambda_expression_untyped = tryGetLambdaFromSQLUserDefinedFunction(function_node.getFunctionName()); - - /** If function is resolved as lambda. - * Clone lambda before resolve. - * Initialize lambda arguments as function arguments - * Resolve lambda and then replace function node with resolved lambda expression body. - * Example: WITH (x -> x + 1) AS lambda SELECT lambda(value) FROM test_table; - * Result: SELECT value + 1 FROM test_table; - */ - if (lambda_expression_untyped) + if (!function_node.isWindowFunction()) { - auto * lambda_expression = lambda_expression_untyped->as(); - if (!lambda_expression) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Function identifier {} must be resolved as lambda. Actual {}. In scope {}", - function_node.getFunctionName(), - lambda_expression_untyped->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); + if (!lambda_expression_untyped) + lambda_expression_untyped = tryGetLambdaFromSQLUserDefinedFunction(function_node.getFunctionName()); - auto lambda_expression_clone = lambda_expression_untyped->clone(); - - IdentifierResolveScope lambda_scope(lambda_expression_clone, &scope /*parent_scope*/); - resolveLambda(lambda_expression_clone, function_arguments, lambda_scope); - - auto & resolved_lambda = lambda_expression_clone->as(); - node = resolved_lambda.getExpression(); - return; - } - - if (function_name == "untuple") - { - /// Special handling of `untuple` function - - if (function_arguments.size() != 1) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Function 'untuple' must have 1 argument. In scope {}", - scope.scope_node->formatASTForErrorMessage()); - - const auto & tuple_argument = function_arguments[0]; - auto result_type = tuple_argument->getResultType(); - const auto * tuple_data_type = typeid_cast(result_type.get()); - if (!tuple_data_type) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Function untuple argument must be have compound type. Actual type {}. In scope {}", - result_type->getName(), - scope.scope_node->formatASTForErrorMessage()); - - const auto & element_names = tuple_data_type->getElementNames(); - - auto result_list = std::make_shared(); - result_list->getNodes().reserve(element_names.size()); - - for (const auto & element_name : element_names) + /** If function is resolved as lambda. + * Clone lambda before resolve. + * Initialize lambda arguments as function arguments. + * Resolve lambda and then replace function node with resolved lambda expression body. + * Example: WITH (x -> x + 1) AS lambda SELECT lambda(value) FROM test_table; + * Result: SELECT value + 1 FROM test_table; + */ + if (lambda_expression_untyped) { - auto tuple_element_function = std::make_shared("tupleElement"); - tuple_element_function->getArguments().getNodes().push_back(tuple_argument); - tuple_element_function->getArguments().getNodes().push_back(std::make_shared(element_name)); + auto * lambda_expression = lambda_expression_untyped->as(); + if (!lambda_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Function identifier {} must be resolved as lambda. Actual {}. In scope {}", + function_node.getFunctionName(), + lambda_expression_untyped->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); - QueryTreeNodePtr function_query_node = tuple_element_function; - resolveFunction(function_query_node, scope); + auto lambda_expression_clone = lambda_expression_untyped->clone(); - if (scope.projection_names_calculation_stage && node->hasAlias()) - scope.node_to_projection_name.emplace(function_query_node, node->getAlias() + '.' + element_name); + IdentifierResolveScope lambda_scope(lambda_expression_clone, &scope /*parent_scope*/); + resolveLambda(lambda_expression_clone, function_arguments, lambda_scope); - result_list->getNodes().push_back(std::move(function_query_node)); + auto & resolved_lambda = lambda_expression_clone->as(); + node = resolved_lambda.getExpression(); + return; } - node = result_list; - return; + if (function_name == "untuple") + { + /// Special handling of `untuple` function + + if (function_arguments.size() != 1) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Function 'untuple' must have 1 argument. In scope {}", + scope.scope_node->formatASTForErrorMessage()); + + const auto & tuple_argument = function_arguments[0]; + auto result_type = tuple_argument->getResultType(); + const auto * tuple_data_type = typeid_cast(result_type.get()); + if (!tuple_data_type) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Function untuple argument must be have compound type. Actual type {}. In scope {}", + result_type->getName(), + scope.scope_node->formatASTForErrorMessage()); + + const auto & element_names = tuple_data_type->getElementNames(); + + auto result_list = std::make_shared(); + result_list->getNodes().reserve(element_names.size()); + + for (const auto & element_name : element_names) + { + auto tuple_element_function = std::make_shared("tupleElement"); + tuple_element_function->getArguments().getNodes().push_back(tuple_argument); + tuple_element_function->getArguments().getNodes().push_back(std::make_shared(element_name)); + + QueryTreeNodePtr function_query_node = tuple_element_function; + resolveFunction(function_query_node, scope); + + if (scope.projection_names_calculation_stage && node->hasAlias()) + scope.node_to_projection_name.emplace(function_query_node, node->getAlias() + '.' + element_name); + + result_list->getNodes().push_back(std::move(function_query_node)); + } + + node = result_list; + return; + } + else if (function_name == "grouping") + { + /// It is responsibility of planner to perform additional handling of grouping function + if (function_arguments_size == 0) + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, + "Function GROUPING expects at least one argument"); + else if (function_arguments_size > 64) + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, + "Function GROUPING can have up to 64 arguments, but {} provided", + function_arguments_size); + + auto grouping_function = std::make_shared(); + auto grouping_function_adaptor = std::make_shared(std::move(grouping_function)); + function_node.resolveAsFunction(std::move(grouping_function_adaptor), std::make_shared()); + return; + } } - else if (function_name == "grouping") + + if (function_node.isWindowFunction()) { - /// It is responsibility of planner to perform additional handling of grouping function - if (function_arguments_size == 0) - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, - "Function GROUPING expects at least one argument"); - else if (function_arguments_size > 64) - throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, - "Function GROUPING can have up to 64 arguments, but {} provided", - function_arguments_size); + if (!AggregateFunctionFactory::instance().isAggregateFunctionName(function_name)) + throw Exception(ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION, + "Aggregate function with name {} does not exists. In scope {}", + function_name, + scope.scope_node->formatASTForErrorMessage()); - auto grouping_function = std::make_shared(); - auto grouping_function_adaptor = std::make_shared(std::move(grouping_function)); - function_node.resolveAsFunction(std::move(grouping_function_adaptor), std::make_shared()); + AggregateFunctionProperties properties; + auto aggregate_function = AggregateFunctionFactory::instance().get(function_name, argument_types, parameters, properties); + + function_node.resolveAsWindowFunction(aggregate_function, aggregate_function->getReturnType()); + resolveWindow(function_node.getWindowNode(), scope); return; } - /** Try to resolve function as - * 1. Executable user defined function. - * 2. Aggregate function. - * 3. Non aggregate function. - * TODO: Provide better hints. - */ FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, context, parameters); if (!function) @@ -2951,7 +3156,7 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc if (!function) { if (!AggregateFunctionFactory::instance().isAggregateFunctionName(function_name)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, + throw Exception(ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION, "Function with name {} does not exists. In scope {}", function_name, scope.scope_node->formatASTForErrorMessage()); @@ -3287,6 +3492,13 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } + case QueryTreeNodeType::WINDOW: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Window {} is not allowed in expression. In scope {}", + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } case QueryTreeNodeType::TABLE: { if (!allow_table_expression) @@ -3467,6 +3679,15 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & sort_co } } +/** Resolve window nodes list. + */ +void QueryAnalyzer::resolveWindowNodeList(QueryTreeNodePtr & window_node_list, IdentifierResolveScope & scope) +{ + auto & window_node_list_typed = window_node_list->as(); + for (auto & node : window_node_list_typed.getNodes()) + resolveWindow(node, scope); +} + class SubqueryToProjectionNameMatcher { public: @@ -4436,10 +4657,10 @@ void assertNoGroupingFunction(const QueryTreeNodePtr & node, const String & asse * 3. Resolve FROM section. * 4. Resolve projection columns. * 5. Resolve expressions in other query parts. - * 6. Remove WITH section from query. - * 7. Validate nodes with duplicate aliases. - * 8. Remove aliases from expression and lambda nodes. - * 9. Validate aggregates. + * 6. Validate nodes with duplicate aliases. + * 7. Validate aggregates, aggregate functions, GROUPING function, window functions. + * 8. Remove WITH and WINDOW sections from query. + * 9. Remove aliases from expression and lambda nodes. * 10. Resolve query tree node with projection columns. */ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope) @@ -4475,6 +4696,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasHaving()) visitor.visit(query_node_typed.getHaving()); + if (query_node_typed.hasWindow()) + visitor.visit(query_node_typed.getWindowNode()); + if (query_node_typed.hasOrderBy()) visitor.visit(query_node_typed.getOrderByNode()); @@ -4521,6 +4745,26 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier return subquery_node && subquery_node->isCTE(); }); + for (auto & window_node : query_node_typed.getWindow().getNodes()) + { + auto & window_node_typed = window_node->as(); + auto parent_window_name = window_node_typed.getParentWindowName(); + if (!parent_window_name.empty()) + { + auto window_node_it = scope.window_name_to_window_node.find(parent_window_name); + if (window_node_it == scope.window_name_to_window_node.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Window '{}' does not exists. In scope {}", + parent_window_name, + scope.scope_node->formatASTForErrorMessage()); + + mergeWindowWithParentWindow(window_node, window_node_it->second, scope); + window_node_typed.setParentWindowName({}); + } + + scope.window_name_to_window_node.emplace(window_node_typed.getAlias(), window_node); + } + /** Disable identifier cache during JOIN TREE resolve. * Depending on JOIN expression section, identifier with same name * can be resolved in different columns. @@ -4579,6 +4823,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasHaving()) resolveExpressionNode(query_node_typed.getHaving(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + if (query_node_typed.hasWindow()) + resolveWindowNodeList(query_node_typed.getWindowNode(), scope); + if (query_node_typed.hasOrderBy()) resolveSortColumnsNodeList(query_node_typed.getOrderByNode(), scope); @@ -4612,13 +4859,6 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier validateLimitOffsetExpression(query_node_typed.getOffset(), "OFFSET", scope); } - /** WITH section can be safely removed, because WITH section only can provide aliases to query expressions - * and CTE for other sections to use. - * - * Example: WITH 1 AS constant, (x -> x + 1) AS lambda, a AS (SELECT * FROM test_table); - */ - query_node_typed.getWithNode() = std::make_shared(); - /** Resolve nodes with duplicate aliases. * Table expressions cannot have duplicate aliases. * @@ -4672,51 +4912,56 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier node->removeAlias(); } - /// Remove aliases from expression and lambda nodes - - for (auto & [_, node] : scope.alias_name_to_expression_node) - node->removeAlias(); - - for (auto & [_, node] : scope.alias_name_to_lambda_node) - node->removeAlias(); - /** Validate aggregates * - * 1. Check that there are no aggregate functions in WHERE. - * 2. Check that there are no aggregate functions in PREWHERE. - * 3. Check that there are no aggregate functions in another aggregate functions. - * 4. Check that there are no columns that are not specified in GROUP BY keys. - * 5. Validate GROUP BY modifiers. + * 1. Check that there are no aggregate functions and GROUPING function in WHERE, in PREWHERE, in another aggregate functions. + * 2. Check that there are no window functions in WHERE, in PREWHERE, in HAVING, in WINDOW, inside another aggregate function, + * inside window function arguments, inside window function window definition. + * 3. Check that there are no columns that are not specified in GROUP BY keys. + * 4. Validate GROUP BY modifiers. */ if (query_node_typed.hasWhere()) { assertNoAggregateFunctionNodes(query_node_typed.getWhere(), "in WHERE"); assertNoGroupingFunction(query_node_typed.getWhere(), "in WHERE"); + assertNoWindowFunctionNodes(query_node_typed.getWhere(), "in WHERE"); } if (query_node_typed.hasPrewhere()) { assertNoAggregateFunctionNodes(query_node_typed.getPrewhere(), "in PREWHERE"); assertNoGroupingFunction(query_node_typed.getPrewhere(), "in PREWHERE"); + assertNoWindowFunctionNodes(query_node_typed.getPrewhere(), "in PREWHERE"); } - QueryTreeNodes aggregate_function_nodes; - if (query_node_typed.hasHaving()) - collectAggregateFunctionNodes(query_node_typed.getHaving(), aggregate_function_nodes); + assertNoWindowFunctionNodes(query_node_typed.getHaving(), "in HAVING"); - if (query_node_typed.hasOrderBy()) - collectAggregateFunctionNodes(query_node_typed.getOrderByNode(), aggregate_function_nodes); + if (query_node_typed.hasWindow()) + assertNoWindowFunctionNodes(query_node_typed.getWindowNode(), "in WINDOW"); - collectAggregateFunctionNodes(query_node_typed.getProjectionNode(), aggregate_function_nodes); + QueryTreeNodes aggregate_function_nodes; + QueryTreeNodes window_function_nodes; + + collectAggregateFunctionNodes(query_node, aggregate_function_nodes); + collectWindowFunctionNodes(query_node, window_function_nodes); for (auto & aggregate_function_node : aggregate_function_nodes) { - for (auto & aggregate_function_node_child : aggregate_function_node->getChildren()) - { - assertNoAggregateFunctionNodes(aggregate_function_node_child, "inside another aggregate function"); - assertNoGroupingFunction(aggregate_function_node_child, "inside another aggregate function"); - } + auto & aggregate_function_node_typed = aggregate_function_node->as(); + + assertNoAggregateFunctionNodes(aggregate_function_node_typed.getArgumentsNode(), "inside another aggregate function"); + assertNoGroupingFunction(aggregate_function_node_typed.getArgumentsNode(), "inside another aggregate function"); + assertNoWindowFunctionNodes(aggregate_function_node_typed.getArgumentsNode(), "inside an aggregate function"); + } + + for (auto & window_function_node : window_function_nodes) + { + auto & window_function_node_typed = window_function_node->as(); + assertNoWindowFunctionNodes(window_function_node_typed.getArgumentsNode(), "inside another window function"); + + if (query_node_typed.hasWindow()) + assertNoWindowFunctionNodes(window_function_node_typed.getWindowNode(), "inside window definition"); } QueryTreeNodes group_by_keys_nodes; @@ -4793,6 +5038,27 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasHaving() && query_node_typed.isGroupByWithTotals() && is_rollup_or_cube) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING"); + /** WITH section can be safely removed, because WITH section only can provide aliases to query expressions + * and CTE for other sections to use. + * + * Example: WITH 1 AS constant, (x -> x + 1) AS lambda, a AS (SELECT * FROM test_table); + */ + query_node_typed.getWith().getNodes().clear(); + + /** WINDOW section can be safely removed, because WINDOW section can only provide window definition to window functions. + * + * Example: SELECT count(*) OVER w FROM test_table WINDOW w AS (PARTITION BY id); + */ + query_node_typed.getWindow().getNodes().clear(); + + /// Remove aliases from expression and lambda nodes + + for (auto & [_, node] : scope.alias_name_to_expression_node) + node->removeAlias(); + + for (auto & [_, node] : scope.alias_name_to_lambda_node) + node->removeAlias(); + query_node_typed.resolveProjectionColumns(std::move(projection_columns)); } diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index 8ebf634e9d8..afe73b7dd80 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -33,6 +33,7 @@ namespace DB * * Constness of function parameters. * Constness of LIMIT and OFFSET. + * Window functions frame. Constness of window functions frame begin OFFSET, end OFFSET. * In SELECT, ORDER BY only columns that are specified in GROUP BY keys after GROUP BY are used. * GROUPING function arguments are specified in GROUP BY keys. * No GROUPING function if there is no GROUP BY. diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 64f3a4561ab..dcababb6296 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -28,6 +28,7 @@ QueryNode::QueryNode() children[with_child_index] = std::make_shared(); children[projection_child_index] = std::make_shared(); children[group_by_child_index] = std::make_shared(); + children[window_child_index] = std::make_shared(); children[order_by_child_index] = std::make_shared(); children[limit_by_child_index] = std::make_shared(); } @@ -75,6 +76,12 @@ String QueryNode::getName() const buffer << getHaving()->getName(); } + if (hasWindow()) + { + buffer << " WINDOW "; + buffer << getWindow().getName(); + } + if (hasOrderBy()) { buffer << " ORDER BY "; @@ -132,9 +139,16 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << ", is_distinct: " << is_distinct; buffer << ", is_limit_with_ties: " << is_limit_with_ties; buffer << ", is_group_by_with_totals: " << is_group_by_with_totals; - buffer << ", is_group_by_with_rollup: " << is_group_by_with_rollup; - buffer << ", is_group_by_with_cube: " << is_group_by_with_cube; - buffer << ", is_group_by_with_grouping_sets: " << is_group_by_with_grouping_sets; + + std::string group_by_type = "ordinary"; + if (is_group_by_with_rollup) + group_by_type = "rollup"; + else if (is_group_by_with_cube) + group_by_type = "cube"; + else if (is_group_by_with_grouping_sets) + group_by_type = "grouping_sets"; + + buffer << ", group_by_type: " << group_by_type; if (!cte_name.empty()) buffer << ", cte_name: " << cte_name; @@ -206,6 +220,12 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s getHaving()->dumpTreeImpl(buffer, format_state, indent + 4); } + if (hasWindow()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "WINDOW\n"; + getWindow().dumpTreeImpl(buffer, format_state, indent + 4); + } + if (hasOrderBy()) { buffer << '\n' << std::string(indent + 2, ' ') << "ORDER BY\n"; @@ -339,6 +359,9 @@ ASTPtr QueryNode::toASTImpl() const if (hasHaving()) select_query->setExpression(ASTSelectQuery::Expression::HAVING, getHaving()->toAST()); + if (hasWindow()) + select_query->setExpression(ASTSelectQuery::Expression::WINDOW, getWindow().toAST()); + if (hasOrderBy()) select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, getOrderBy().toAST()); diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 6d7c4c2792e..821df4d666e 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -189,6 +189,31 @@ public: return children[join_tree_child_index]; } + bool hasWindow() const + { + return !getWindow().getNodes().empty(); + } + + const ListNode & getWindow() const + { + return children[window_child_index]->as(); + } + + ListNode & getWindow() + { + return children[window_child_index]->as(); + } + + const QueryTreeNodePtr & getWindowNode() const + { + return children[window_child_index]; + } + + QueryTreeNodePtr & getWindowNode() + { + return children[window_child_index]; + } + bool hasPrewhere() const { return children[prewhere_child_index] != nullptr; @@ -453,13 +478,14 @@ private: static constexpr size_t where_child_index = 4; static constexpr size_t group_by_child_index = 5; static constexpr size_t having_child_index = 6; - static constexpr size_t order_by_child_index = 7; - static constexpr size_t interpolate_child_index = 8; - static constexpr size_t limit_by_limit_child_index = 9; - static constexpr size_t limit_by_offset_child_index = 10; - static constexpr size_t limit_by_child_index = 11; - static constexpr size_t limit_child_index = 12; - static constexpr size_t offset_child_index = 13; + static constexpr size_t window_child_index = 7; + static constexpr size_t order_by_child_index = 8; + static constexpr size_t interpolate_child_index = 9; + static constexpr size_t limit_by_limit_child_index = 10; + static constexpr size_t limit_by_offset_child_index = 11; + static constexpr size_t limit_by_child_index = 12; + static constexpr size_t limit_child_index = 13; + static constexpr size_t offset_child_index = 14; static constexpr size_t children_size = offset_child_index + 1; }; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 168203f3f91..b88215606cc 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include @@ -34,6 +35,7 @@ #include #include #include +#include #include #include #include @@ -84,10 +86,14 @@ private: QueryTreeNodePtr buildInterpolateColumnList(const ASTPtr & interpolate_expression_list) const; + QueryTreeNodePtr buildWindowList(const ASTPtr & window_definition_list) const; + QueryTreeNodePtr buildExpressionList(const ASTPtr & expression_list) const; QueryTreeNodePtr buildExpression(const ASTPtr & expression) const; + QueryTreeNodePtr buildWindow(const ASTPtr & window_definition) const; + QueryTreeNodePtr buildJoinTree(const ASTPtr & tables_in_select_query) const; ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const; @@ -250,6 +256,10 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q if (having_expression) current_query_tree->getHaving() = buildExpression(having_expression); + auto window_list = select_query_typed.window(); + if (window_list) + current_query_tree->getWindowNode() = buildWindowList(window_list); + auto select_order_by_list = select_query_typed.orderBy(); if (select_order_by_list) current_query_tree->getOrderByNode() = buildSortColumnList(select_order_by_list); @@ -342,6 +352,26 @@ QueryTreeNodePtr QueryTreeBuilder::buildInterpolateColumnList(const ASTPtr & int return list_node; } +QueryTreeNodePtr QueryTreeBuilder::buildWindowList(const ASTPtr & window_definition_list) const +{ + auto list_node = std::make_shared(); + + auto & expression_list_typed = window_definition_list->as(); + list_node->getNodes().reserve(expression_list_typed.children.size()); + + for (auto & window_list_element : expression_list_typed.children) + { + const auto & window_list_element_typed = window_list_element->as(); + + auto window_node = buildWindow(window_list_element_typed.definition); + window_node->setAlias(window_list_element_typed.name); + + list_node->getNodes().push_back(std::move(window_node)); + } + + return list_node; +} + QueryTreeNodePtr QueryTreeBuilder::buildExpressionList(const ASTPtr & expression_list) const { auto list_node = std::make_shared(); @@ -451,6 +481,14 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co function_node->getArguments().getNodes().push_back(buildExpression(argument)); } + if (function->is_window_function) + { + if (function->window_definition) + function_node->getWindowNode() = buildWindow(function->window_definition); + else + function_node->getWindowNode() = std::make_shared(Identifier(function->window_name)); + } + result = function_node; } } @@ -522,6 +560,41 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co return result; } +QueryTreeNodePtr QueryTreeBuilder::buildWindow(const ASTPtr & window_definition) const +{ + const auto & window_definition_typed = window_definition->as(); + WindowFrame window_frame; + + if (!window_definition_typed.frame_is_default) + { + window_frame.is_default = false; + window_frame.type = window_definition_typed.frame_type; + window_frame.begin_type = window_definition_typed.frame_begin_type; + window_frame.begin_preceding = window_definition_typed.frame_begin_preceding; + window_frame.end_type = window_definition_typed.frame_end_type; + window_frame.end_preceding = window_definition_typed.frame_end_preceding; + } + + auto window_node = std::make_shared(window_frame); + window_node->setParentWindowName(window_definition_typed.parent_window_name); + + if (window_definition_typed.partition_by) + window_node->getPartitionByNode() = buildExpressionList(window_definition_typed.partition_by); + + if (window_definition_typed.order_by) + window_node->getOrderByNode() = buildSortColumnList(window_definition_typed.order_by); + + if (window_definition_typed.frame_begin_offset) + window_node->getFrameBeginOffsetNode() = buildExpression(window_definition_typed.frame_begin_offset); + + if (window_definition_typed.frame_end_offset) + window_node->getFrameEndOffsetNode() = buildExpression(window_definition_typed.frame_end_offset); + + window_node->setOriginalAST(window_definition); + + return window_node; +} + QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select_query) const { if (!tables_in_select_query) diff --git a/src/Analyzer/WindowNode.cpp b/src/Analyzer/WindowNode.cpp new file mode 100644 index 00000000000..c4d11153a31 --- /dev/null +++ b/src/Analyzer/WindowNode.cpp @@ -0,0 +1,207 @@ +#include + +#include + +#include +#include + +#include + +namespace DB +{ + +WindowNode::WindowNode(WindowFrame window_frame_) + : window_frame(std::move(window_frame_)) +{ + children.resize(children_size); + children[partition_by_child_index] = std::make_shared(); + children[order_by_child_index] = std::make_shared(); +} + +String WindowNode::getName() const +{ + String result; + + if (hasPartitionBy()) + { + result += "PARTITION BY"; + result += getPartitionBy().getName(); + } + + if (hasOrderBy()) + { + result += "ORDER BY"; + result += getOrderBy().getName(); + } + + if (!window_frame.is_default) + { + if (hasPartitionBy() || hasOrderBy()) + result += ' '; + + if (window_frame.type == WindowFrame::FrameType::ROWS) + result += "ROWS"; + else if (window_frame.type == WindowFrame::FrameType::GROUPS) + result += "GROUPS"; + else if (window_frame.type == WindowFrame::FrameType::RANGE) + result += "RANGE"; + + result += " BETWEEN "; + if (window_frame.begin_type == WindowFrame::BoundaryType::Current) + { + result += "CURRENT ROW"; + } + else if (window_frame.begin_type == WindowFrame::BoundaryType::Unbounded) + { + result += "UNBOUNDED"; + result += " "; + result += (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); + } + else + { + result += getFrameBeginOffsetNode()->getName(); + result += " "; + result += (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); + } + + result += " AND "; + + if (window_frame.end_type == WindowFrame::BoundaryType::Current) + { + result += "CURRENT ROW"; + } + else if (window_frame.end_type == WindowFrame::BoundaryType::Unbounded) + { + result += "UNBOUNDED"; + result += " "; + result += (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); + } + else + { + result += getFrameEndOffsetNode()->getName(); + result += " "; + result += (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); + } + } + + return result; +} + +void WindowNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const +{ + buffer << std::string(indent, ' ') << "WINDOW id: " << format_state.getNodeId(this); + + if (hasAlias()) + buffer << ", alias: " << getAlias(); + + buffer << ", parent_window_name: " << parent_window_name; + buffer << ", frame_type: " << window_frame.type; + + auto window_frame_bound_type_to_string = [](WindowFrame::BoundaryType boundary_type, bool boundary_preceding) + { + std::string value; + + if (boundary_type == WindowFrame::BoundaryType::Unbounded) + value = "unbounded"; + else if (boundary_type == WindowFrame::BoundaryType::Current) + value = "current"; + else if (boundary_type == WindowFrame::BoundaryType::Offset) + value = "offset"; + + if (boundary_type != WindowFrame::BoundaryType::Current) + { + if (boundary_preceding) + value += " preceding"; + else + value += " following"; + } + + return value; + }; + + buffer << ", frame_begin_type: " << window_frame_bound_type_to_string(window_frame.begin_type, window_frame.begin_preceding); + buffer << ", frame_end_type: " << window_frame_bound_type_to_string(window_frame.end_type, window_frame.end_preceding); + + if (hasPartitionBy()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "PARTITION BY\n"; + getPartitionBy().dumpTreeImpl(buffer, format_state, indent + 4); + } + + if (hasOrderBy()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "ORDER BY\n"; + getOrderBy().dumpTreeImpl(buffer, format_state, indent + 4); + } + + if (hasFrameBeginOffset()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "FRAME BEGIN OFFSET\n"; + getFrameBeginOffsetNode()->dumpTreeImpl(buffer, format_state, indent + 4); + } + + if (hasFrameEndOffset()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "FRAME END OFFSET\n"; + getFrameEndOffsetNode()->dumpTreeImpl(buffer, format_state, indent + 4); + } +} + +bool WindowNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + + return window_frame == rhs_typed.window_frame && parent_window_name == rhs_typed.parent_window_name; +} + +void WindowNode::updateTreeHashImpl(HashState & hash_state) const +{ + hash_state.update(window_frame.is_default); + hash_state.update(window_frame.type); + hash_state.update(window_frame.begin_type); + hash_state.update(window_frame.begin_preceding); + hash_state.update(window_frame.end_type); + hash_state.update(window_frame.end_preceding); +} + +ASTPtr WindowNode::toASTImpl() const +{ + auto window_definition = std::make_shared(); + + window_definition->children.push_back(getPartitionByNode()->toAST()); + window_definition->partition_by = window_definition->children.back(); + + window_definition->children.push_back(getOrderByNode()->toAST()); + window_definition->order_by = window_definition->children.back(); + + window_definition->frame_is_default = window_frame.is_default; + window_definition->frame_type = window_frame.type; + window_definition->frame_begin_type = window_frame.begin_type; + window_definition->frame_begin_preceding = window_frame.begin_preceding; + + if (hasFrameBeginOffset()) + { + window_definition->children.push_back(getFrameBeginOffsetNode()->toAST()); + window_definition->frame_begin_offset = window_definition->children.back(); + } + + window_definition->frame_end_type = window_frame.end_type; + window_definition->frame_end_preceding = window_frame.end_preceding; + if (hasFrameEndOffset()) + { + window_definition->children.push_back(getFrameEndOffsetNode()->toAST()); + window_definition->frame_end_offset = window_definition->children.back(); + } + + return window_definition; +} + +QueryTreeNodePtr WindowNode::cloneImpl() const +{ + auto window_node = std::make_shared(window_frame); + window_node->parent_window_name = parent_window_name; + + return window_node; +} + +} diff --git a/src/Analyzer/WindowNode.h b/src/Analyzer/WindowNode.h new file mode 100644 index 00000000000..e99b66088ca --- /dev/null +++ b/src/Analyzer/WindowNode.h @@ -0,0 +1,193 @@ +#pragma once + +#include +#include + +#include + +namespace DB +{ + +/** Window node represents window function window description. + * + * Example: SELECT * FROM test_table WINDOW window AS (PARTITION BY id); + * window AS (PARTITION BY id) - window node. + * + * Example: SELECT count() OVER (PARTITION BY id) FROM test_table; + * PARTITION BY id - window node. + * + * Window node can also refer to its parent window node. + * Example: SELECT count() OVER (parent_window ORDER BY id) FROM test_table WINDOW parent_window AS (PARTITION BY id); + * parent_window ORDER BY id - window node. + * + * Window node initially initialized with window frame. + * + * If window frame has OFFSET begin type, additionally frame begin offset node must be initialized. + * If window frame has OFFSET end type, additionally frame end offset node must be initialized. + * During query analysis pass they must be resolved, validated and window node window frame offset constants must be updated. + */ +class WindowNode; +using WindowNodePtr = std::shared_ptr; + +class WindowNode final : public IQueryTreeNode +{ +public: + /// Initialize window node with window frame + explicit WindowNode(WindowFrame window_frame_); + + /// Get window node window frame + const WindowFrame & getWindowFrame() const + { + return window_frame; + } + + /// Get window node window frame + WindowFrame & getWindowFrame() + { + return window_frame; + } + + /// Has parent window name + bool hasParentWindowName() const + { + return parent_window_name.empty(); + } + + /// Get parent window name + const String & getParentWindowName() const + { + return parent_window_name; + } + + /// Set parent window name + void setParentWindowName(String parent_window_name_value) + { + parent_window_name = std::move(parent_window_name_value); + } + + /// Has order by + bool hasOrderBy() const + { + return !getOrderBy().getNodes().empty(); + } + + /// Get order by + const ListNode & getOrderBy() const + { + return children[order_by_child_index]->as(); + } + + /// Get order by + ListNode & getOrderBy() + { + return children[order_by_child_index]->as(); + } + + /// Get order by + const QueryTreeNodePtr & getOrderByNode() const + { + return children[order_by_child_index]; + } + + /// Get order by + QueryTreeNodePtr & getOrderByNode() + { + return children[order_by_child_index]; + } + + /// Has partition by + bool hasPartitionBy() const + { + return !getPartitionBy().getNodes().empty(); + } + + /// Get partition by + const ListNode & getPartitionBy() const + { + return children[partition_by_child_index]->as(); + } + + /// Get partition by + ListNode & getPartitionBy() + { + return children[partition_by_child_index]->as(); + } + + /// Get partition by + const QueryTreeNodePtr & getPartitionByNode() const + { + return children[partition_by_child_index]; + } + + /// Get partition by + QueryTreeNodePtr & getPartitionByNode() + { + return children[partition_by_child_index]; + } + + /// Has frame begin offset + bool hasFrameBeginOffset() const + { + return getFrameBeginOffsetNode() != nullptr; + } + + /// Get FRAME begin offset node + const QueryTreeNodePtr & getFrameBeginOffsetNode() const + { + return children[frame_begin_offset_child_index]; + } + + /// Get FRAME begin offset node + QueryTreeNodePtr & getFrameBeginOffsetNode() + { + return children[frame_begin_offset_child_index]; + } + + /// Has frame end offset + bool hasFrameEndOffset() const + { + return getFrameEndOffsetNode() != nullptr; + } + + /// Get FRAME begin offset node + const QueryTreeNodePtr & getFrameEndOffsetNode() const + { + return children[frame_end_offset_child_index]; + } + + /// Get FRAME begin offset node + QueryTreeNodePtr & getFrameEndOffsetNode() + { + return children[frame_end_offset_child_index]; + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::WINDOW; + } + + String getName() const override; + + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + + void updateTreeHashImpl(HashState & hash_state) const override; + +protected: + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + static constexpr size_t order_by_child_index = 0; + static constexpr size_t partition_by_child_index = 1; + static constexpr size_t frame_begin_offset_child_index = 3; + static constexpr size_t frame_end_offset_child_index = 4; + static constexpr size_t children_size = frame_end_offset_child_index + 1; + + WindowFrame window_frame; + String parent_window_name; +}; + +} diff --git a/src/Interpreters/WindowDescription.cpp b/src/Interpreters/WindowDescription.cpp index 335610b2be9..7ed7788cf1d 100644 --- a/src/Interpreters/WindowDescription.cpp +++ b/src/Interpreters/WindowDescription.cpp @@ -20,7 +20,8 @@ std::string WindowFunctionDescription::dump() const WriteBufferFromOwnString ss; ss << "window function '" << column_name << "\n"; - ss << "function node " << function_node->dumpTree() << "\n"; + if (function_node) + ss << "function node " << function_node->dumpTree() << "\n"; ss << "aggregate function '" << aggregate_function->getName() << "'\n"; if (!function_parameters.empty()) { diff --git a/src/Interpreters/WindowDescription.h b/src/Interpreters/WindowDescription.h index e7bc0473c26..3b9af6575e8 100644 --- a/src/Interpreters/WindowDescription.h +++ b/src/Interpreters/WindowDescription.h @@ -99,7 +99,6 @@ struct WindowDescription // The window functions that are calculated for this window. std::vector window_functions; - std::string dump() const; void checkValid() const; diff --git a/src/Planner/ActionsChain.cpp b/src/Planner/ActionsChain.cpp index d300b688389..21f663e1a8a 100644 --- a/src/Planner/ActionsChain.cpp +++ b/src/Planner/ActionsChain.cpp @@ -33,6 +33,8 @@ void ActionsChainStep::finalizeInputAndOutputColumns(const NameSet & child_input actions->addOrReplaceInOutputs(*required_output_node); actions->removeUnusedActions(); + /// TODO: Analyzer fix ActionsDAG input and constant nodes with same name + actions->projectInput(); initialize(); } @@ -69,15 +71,30 @@ void ActionsChainStep::initialize() available_output_columns.clear(); + /// TODO: Analyzer fix ActionsDAG input and constant nodes with same name + std::unordered_set available_output_columns_names; + if (available_output_columns_strategy == AvailableOutputColumnsStrategy::ALL_NODES) { for (const auto & node : actions->getNodes()) + { + if (available_output_columns_names.contains(node.result_name)) + continue; + available_output_columns.emplace_back(node.column, node.result_type, node.result_name); + available_output_columns_names.insert(node.result_name); + } } else if (available_output_columns_strategy == AvailableOutputColumnsStrategy::OUTPUT_NODES) { for (const auto & node : actions->getOutputs()) + { + if (available_output_columns_names.contains(node->result_name)) + continue; + available_output_columns.emplace_back(node->column, node->result_type, node->result_name); + available_output_columns_names.insert(node->result_name); + } } available_output_columns.insert(available_output_columns.end(), additional_output_columns.begin(), additional_output_columns.end()); diff --git a/src/Planner/ActionsChain.h b/src/Planner/ActionsChain.h index 9a4c003c1ea..89f8ad2a319 100644 --- a/src/Planner/ActionsChain.h +++ b/src/Planner/ActionsChain.h @@ -65,10 +65,10 @@ public: explicit ActionsChainStep(ActionsDAGPtr actions_, AvailableOutputColumnsStrategy available_output_columns_stategy_, - const ColumnsWithTypeAndName & additional_output_columns_) + ColumnsWithTypeAndName additional_output_columns_) : actions(std::move(actions_)) , available_output_columns_strategy(available_output_columns_stategy_) - , additional_output_columns(additional_output_columns_) + , additional_output_columns(std::move(additional_output_columns_)) { initialize(); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d300db74334..0ad1c693339 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include @@ -47,6 +48,7 @@ #include #include #include +#include #include #include #include @@ -56,7 +58,8 @@ #include #include #include -#include +#include +#include #include #include @@ -64,6 +67,7 @@ #include #include #include +#include #include #include #include @@ -87,7 +91,6 @@ namespace ErrorCodes * TODO: Support distributed query processing * TODO: Support PREWHERE * TODO: Support ORDER BY - * TODO: Support WINDOW FUNCTIONS * TODO: Support DISTINCT * TODO: Support trivial count optimization * TODO: Support projections @@ -473,18 +476,22 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, auto result_plan = QueryPlan(); result_plan.unitePlans(std::move(join_step), {std::move(plans)}); - auto drop_unused_columns_after_join = std::make_shared(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + auto drop_unused_columns_after_join_actions_dag = std::make_shared(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs updated_outputs; + std::unordered_set updated_outputs_names; - for (auto & output : drop_unused_columns_after_join->getOutputs()) + for (auto & output : drop_unused_columns_after_join_actions_dag->getOutputs()) { - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output->result_name)) - updated_outputs.push_back(output); + if (updated_outputs_names.contains(output->result_name) || !planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output->result_name)) + continue; + + updated_outputs.push_back(output); + updated_outputs_names.insert(output->result_name); } - drop_unused_columns_after_join->getOutputs() = std::move(updated_outputs); + drop_unused_columns_after_join_actions_dag->getOutputs() = std::move(updated_outputs); - auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentDataStream(), std::move(drop_unused_columns_after_join)); + auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentDataStream(), std::move(drop_unused_columns_after_join_actions_dag)); drop_unused_columns_after_join_transform_step->setStepDescription("DROP unused columns after JOIN"); result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); @@ -783,7 +790,7 @@ void Planner::buildQueryPlanIfNeeded() where_action_step_index = actions_chain.getLastStepIndex(); } - auto aggregate_function_nodes = extractAggregateFunctionNodes(query_tree); + auto aggregate_function_nodes = collectAggregateFunctionNodes(query_tree); AggregateDescriptions aggregates_descriptions = extractAggregateDescriptions(aggregate_function_nodes, *planner_context); ColumnsWithTypeAndName aggregates_columns; aggregates_columns.reserve(aggregates_descriptions.size()); @@ -935,6 +942,83 @@ void Planner::buildQueryPlanIfNeeded() having_action_step_index = actions_chain.getLastStepIndex(); } + auto window_function_nodes = collectWindowFunctionNodes(query_tree); + auto window_descriptions = extractWindowDescriptions(window_function_nodes, *planner_context); + std::optional before_window_step_index; + + if (!window_function_nodes.empty()) + { + chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & window_input = chain_available_output_columns ? *chain_available_output_columns + : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + ActionsDAGPtr before_window_actions_dag = std::make_shared(window_input); + before_window_actions_dag->getOutputs().clear(); + + std::unordered_set before_window_actions_dag_output_nodes_names; + + for (auto & window_function_node : window_function_nodes) + { + auto & window_function_node_typed = window_function_node->as(); + auto & window_node = window_function_node_typed.getWindowNode()->as(); + + auto expression_dag_nodes = actions_visitor.visit(before_window_actions_dag, window_function_node_typed.getArgumentsNode()); + aggregation_keys.reserve(expression_dag_nodes.size()); + + for (auto & expression_dag_node : expression_dag_nodes) + { + if (before_window_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) + continue; + + before_window_actions_dag->getOutputs().push_back(expression_dag_node); + before_window_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); + } + + expression_dag_nodes = actions_visitor.visit(before_window_actions_dag, window_node.getPartitionByNode()); + aggregation_keys.reserve(expression_dag_nodes.size()); + + for (auto & expression_dag_node : expression_dag_nodes) + { + if (before_window_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) + continue; + + before_window_actions_dag->getOutputs().push_back(expression_dag_node); + before_window_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); + } + + /** We add only sort column sort expression in before WINDOW actions DAG. + * WITH fill expressions must be constant nodes. + */ + auto & order_by_node_list = window_node.getOrderBy(); + for (auto & sort_column_node : order_by_node_list.getNodes()) + { + auto & sort_column_node_typed = sort_column_node->as(); + expression_dag_nodes = actions_visitor.visit(before_window_actions_dag, sort_column_node_typed.getExpression()); + + for (auto & expression_dag_node : expression_dag_nodes) + { + if (before_window_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) + continue; + + before_window_actions_dag->getOutputs().push_back(expression_dag_node); + before_window_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); + } + } + } + + ColumnsWithTypeAndName window_functions_additional_columns; + + for (auto & window_description : window_descriptions) + for (auto & window_function : window_description.window_functions) + window_functions_additional_columns.emplace_back(nullptr, window_function.aggregate_function->getReturnType(), window_function.column_name); + + auto before_window_step = std::make_unique(std::move(before_window_actions_dag), + ActionsChainStep::AvailableOutputColumnsStrategy::ALL_NODES, + window_functions_additional_columns); + actions_chain.addStep(std::move(before_window_step)); + before_window_step_index = actions_chain.getLastStepIndex(); + } + chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), projection_input, planner_context); @@ -1196,6 +1280,53 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(having_step)); } + if (before_window_step_index) + { + auto & before_window_actions_chain_node = actions_chain.at(*before_window_step_index); + auto expression_step_before_window = std::make_unique(query_plan.getCurrentDataStream(), + before_window_actions_chain_node->getActions()); + expression_step_before_window->setStepDescription("Before WINDOW"); + query_plan.addStep(std::move(expression_step_before_window)); + + sortWindowDescriptions(window_descriptions); + size_t window_descriptions_size = window_descriptions.size(); + + const auto & settings = query_context->getSettingsRef(); + for (size_t i = 0; i < window_descriptions_size; ++i) + { + const auto & window_description = window_descriptions[i]; + + /** We don't need to sort again if the input from previous window already + * has suitable sorting. Also don't create sort steps when there are no + * columns to sort by, because the sort nodes are confused by this. It + * happens in case of `over ()`. + */ + if (!window_description.full_sort_description.empty() && + (i == 0 || !sortDescriptionIsPrefix(window_description.full_sort_description, window_descriptions[i - 1].full_sort_description))) + { + auto sorting_step = std::make_unique( + query_plan.getCurrentDataStream(), + window_description.full_sort_description, + settings.max_block_size, + 0 /* LIMIT */, + SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), + settings.max_bytes_before_remerge_sort, + settings.remerge_sort_lowered_memory_bytes_ratio, + settings.max_bytes_before_external_sort, + query_context->getTemporaryVolume(), + settings.min_free_disk_space_for_temporary_data, + settings.optimize_sorting_by_input_stream_properties); + + sorting_step->setStepDescription("Sorting for window '" + window_description.window_name + "'"); + query_plan.addStep(std::move(sorting_step)); + } + + auto window_step = std::make_unique(query_plan.getCurrentDataStream(), window_description, window_description.window_functions); + window_step->setStepDescription("Window step for window '" + window_description.window_name + "'"); + query_plan.addStep(std::move(window_step)); + } + } + auto & projection_actions_chain_node = actions_chain.at(projection_step_index); auto expression_step_projection = std::make_unique(query_plan.getCurrentDataStream(), projection_actions_chain_node->getActions()); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index bfec9238e83..47754587241 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include #include @@ -419,7 +421,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi auto function_node_name = calculateActionNodeName(node, *planner_context, node_to_node_name); - if (function_node.isAggregateFunction()) + if (function_node.isAggregateFunction() || function_node.isWindowFunction()) { size_t actions_stack_size = actions_stack.size(); @@ -573,6 +575,13 @@ String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerConte buffer << ')'; + if (function_node.isWindowFunction()) + { + buffer << " OVER ("; + buffer << calculateWindowNodeActionName(function_node.getWindowNode(), planner_context, node_to_name); + buffer << ')'; + } + result = buffer.str(); } break; @@ -627,4 +636,119 @@ String calculateConstantActionNodeName(const Field & constant_literal) return calculateConstantActionNodeName(constant_literal, applyVisitor(FieldToDataType(), constant_literal)); } +String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name) +{ + auto & window_node = node->as(); + WriteBufferFromOwnString buffer; + + if (window_node.hasPartitionBy()) + { + buffer << "PARTITION BY "; + + auto & partition_by_nodes = window_node.getPartitionBy().getNodes(); + size_t partition_by_nodes_size = partition_by_nodes.size(); + + for (size_t i = 0; i < partition_by_nodes_size; ++i) + { + auto & partition_by_node = partition_by_nodes[i]; + buffer << calculateActionNodeName(partition_by_node, planner_context, node_to_name); + if (i + 1 != partition_by_nodes_size) + buffer << ", "; + } + } + + if (window_node.hasOrderBy()) + { + if (window_node.hasPartitionBy()) + buffer << ' '; + + buffer << "ORDER BY "; + + auto & order_by_nodes = window_node.getOrderBy().getNodes(); + size_t order_by_nodes_size = order_by_nodes.size(); + + for (size_t i = 0; i < order_by_nodes_size; ++i) + { + auto & sort_column_node = order_by_nodes[i]->as(); + + buffer << calculateActionNodeName(sort_column_node.getExpression(), planner_context, node_to_name); + auto sort_direction = sort_column_node.getSortDirection(); + buffer << (sort_direction == SortDirection::ASCENDING ? " ASC" : " DESC"); + + auto nulls_sort_direction = sort_column_node.getNullsSortDirection(); + + if (nulls_sort_direction) + buffer << " NULLS " << (nulls_sort_direction == sort_direction ? "LAST" : "FIRST"); + + if (auto collator = sort_column_node.getCollator()) + buffer << " COLLATE " << collator->getLocale(); + + if (sort_column_node.withFill()) + { + buffer << " WITH FILL"; + + if (sort_column_node.hasFillFrom()) + buffer << " FROM " << calculateActionNodeName(sort_column_node.getFillFrom(), planner_context, node_to_name); + + if (sort_column_node.hasFillTo()) + buffer << " TO " << calculateActionNodeName(sort_column_node.getFillTo(), planner_context, node_to_name); + + if (sort_column_node.hasFillStep()) + buffer << " STEP " << calculateActionNodeName(sort_column_node.getFillStep(), planner_context, node_to_name); + } + + if (i + 1 != order_by_nodes_size) + buffer << ", "; + } + } + + auto & window_frame = window_node.getWindowFrame(); + if (!window_frame.is_default) + { + if (window_node.hasPartitionBy() || window_node.hasOrderBy()) + buffer << ' '; + + buffer << window_frame.type << " BETWEEN "; + if (window_frame.begin_type == WindowFrame::BoundaryType::Current) + { + buffer << "CURRENT ROW"; + } + else if (window_frame.begin_type == WindowFrame::BoundaryType::Unbounded) + { + buffer << "UNBOUNDED"; + buffer << " " << (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); + } + else + { + buffer << calculateActionNodeName(window_node.getFrameBeginOffsetNode(), planner_context, node_to_name); + buffer << " " << (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); + } + + buffer << " AND "; + + if (window_frame.end_type == WindowFrame::BoundaryType::Current) + { + buffer << "CURRENT ROW"; + } + else if (window_frame.end_type == WindowFrame::BoundaryType::Unbounded) + { + buffer << "UNBOUNDED"; + buffer << " " << (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); + } + else + { + buffer << calculateActionNodeName(window_node.getFrameEndOffsetNode(), planner_context, node_to_name); + buffer << " " << (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); + } + } + + return buffer.str(); +} + +String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context) +{ + QueryTreeNodeToName empty_map; + return calculateWindowNodeActionName(node, planner_context, empty_map); +} + } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 2dc34fb32a1..8cc75461d6c 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -53,7 +53,7 @@ private: using QueryTreeNodeToName = std::unordered_map; String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name); -/** Calculate query tree expression node name action dag name. +/** Calculate query tree expression node action dag name. * * For column node column node identifier from planner context is used. */ @@ -65,4 +65,14 @@ String calculateConstantActionNodeName(const Field & constant_literal, const Dat /// Calculate action node name for constant, data type will be derived from constant literal value String calculateConstantActionNodeName(const Field & constant_literal); +/** Calculate action node name for window node. + * Window node action name can only be part of window function action name. + */ +String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context, QueryTreeNodeToName & node_to_name); + +/** Calculate action node name for window node. + * Window node action name can only be part of window function action name. + */ +String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const PlannerContext & planner_context); + } diff --git a/src/Planner/PlannerAggregation.cpp b/src/Planner/PlannerAggregation.cpp index 8a306fd464c..0d977193351 100644 --- a/src/Planner/PlannerAggregation.cpp +++ b/src/Planner/PlannerAggregation.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include @@ -193,22 +193,6 @@ void resolveGroupingFunctions(QueryTreeNodePtr & query_node, planner_context); } -QueryTreeNodes extractAggregateFunctionNodes(const QueryTreeNodePtr & query_node) -{ - const auto & query_node_typed = query_node->as(); - - QueryTreeNodes aggregate_function_nodes; - if (query_node_typed.hasHaving()) - collectAggregateFunctionNodes(query_node_typed.getHaving(), aggregate_function_nodes); - - if (query_node_typed.hasOrderBy()) - collectAggregateFunctionNodes(query_node_typed.getOrderByNode(), aggregate_function_nodes); - - collectAggregateFunctionNodes(query_node_typed.getProjectionNode(), aggregate_function_nodes); - - return aggregate_function_nodes; -} - AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodes & aggregate_function_nodes, const PlannerContext & planner_context) { QueryTreeNodeToName node_to_name; @@ -217,16 +201,16 @@ AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodes & aggreg for (const auto & aggregate_function_node : aggregate_function_nodes) { - const auto & aggregagte_function_node_typed = aggregate_function_node->as(); + const auto & aggregate_function_node_typed = aggregate_function_node->as(); String node_name = calculateActionNodeName(aggregate_function_node, planner_context, node_to_name); auto [_, inserted] = unique_aggregate_action_node_names.emplace(node_name); if (!inserted) continue; AggregateDescription aggregate_description; - aggregate_description.function = aggregagte_function_node_typed.getAggregateFunction(); + aggregate_description.function = aggregate_function_node_typed.getAggregateFunction(); - const auto & parameters_nodes = aggregagte_function_node_typed.getParameters().getNodes(); + const auto & parameters_nodes = aggregate_function_node_typed.getParameters().getNodes(); aggregate_description.parameters.reserve(parameters_nodes.size()); for (const auto & parameter_node : parameters_nodes) @@ -235,7 +219,7 @@ AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodes & aggreg aggregate_description.parameters.push_back(parameter_node->getConstantValue().getValue()); } - const auto & arguments_nodes = aggregagte_function_node_typed.getArguments().getNodes(); + const auto & arguments_nodes = aggregate_function_node_typed.getArguments().getNodes(); aggregate_description.argument_names.reserve(arguments_nodes.size()); for (const auto & argument_node : arguments_nodes) diff --git a/src/Planner/PlannerAggregation.h b/src/Planner/PlannerAggregation.h index b4fc449400b..6dfd7faca22 100644 --- a/src/Planner/PlannerAggregation.h +++ b/src/Planner/PlannerAggregation.h @@ -20,9 +20,6 @@ void resolveGroupingFunctions(QueryTreeNodePtr & query_node, const GroupingSetsParamsList & grouping_sets_parameters_list, const PlannerContext & planner_context); -/// Extract aggregate function nodes from query node -QueryTreeNodes extractAggregateFunctionNodes(const QueryTreeNodePtr & query_node); - /// Extract aggregate descriptions from aggregate function nodes AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodes & aggregate_function_nodes, const PlannerContext & planner_context); diff --git a/src/Planner/PlannerSorting.h b/src/Planner/PlannerSorting.h index ae3d2d1acfe..c4e4e634973 100644 --- a/src/Planner/PlannerSorting.h +++ b/src/Planner/PlannerSorting.h @@ -7,7 +7,7 @@ namespace DB { -/// Extract sort description from query order by node +/// Extract sort description from order by node SortDescription extractSortDescription(const QueryTreeNodePtr & order_by_node, const PlannerContext & planner_context); } diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp new file mode 100644 index 00000000000..a1ae572727f --- /dev/null +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -0,0 +1,145 @@ +#include + +#include +#include +#include + +#include + +#include +#include + +namespace DB +{ + +namespace +{ + +WindowDescription extractWindowDescriptionFromWindowNode(const QueryTreeNodePtr & node, const PlannerContext & planner_context) +{ + auto & window_node = node->as(); + + WindowDescription window_description; + window_description.window_name = calculateWindowNodeActionName(node, planner_context); + + for (const auto & partition_by_node : window_node.getPartitionBy().getNodes()) + { + auto partition_by_node_action_name = calculateActionNodeName(partition_by_node, planner_context); + auto partition_by_sort_column_description = SortColumnDescription(partition_by_node_action_name, 1 /* direction */, 1 /* nulls_direction */); + window_description.partition_by.push_back(std::move(partition_by_sort_column_description)); + } + + window_description.order_by = extractSortDescription(window_node.getOrderByNode(), planner_context); + + window_description.full_sort_description = window_description.partition_by; + window_description.full_sort_description.insert(window_description.full_sort_description.end(), window_description.order_by.begin(), window_description.order_by.end()); + + /// WINDOW frame is validated during query analysis stage + window_description.frame = window_node.getWindowFrame(); + + const auto & query_context = planner_context.getQueryContext(); + const auto & query_context_settings = query_context->getSettingsRef(); + + bool compile_sort_description = query_context_settings.compile_sort_description; + size_t min_count_to_compile_sort_description = query_context_settings.min_count_to_compile_sort_description; + + window_description.partition_by.compile_sort_description = compile_sort_description; + window_description.partition_by.min_count_to_compile_sort_description = min_count_to_compile_sort_description; + + window_description.order_by.compile_sort_description = compile_sort_description; + window_description.order_by.min_count_to_compile_sort_description = min_count_to_compile_sort_description; + + window_description.full_sort_description.compile_sort_description = compile_sort_description; + window_description.full_sort_description.min_count_to_compile_sort_description = min_count_to_compile_sort_description; + + return window_description; +} + +} + +std::vector extractWindowDescriptions(const QueryTreeNodes & window_function_nodes, const PlannerContext & planner_context) +{ + std::unordered_map window_name_to_description; + + for (const auto & window_function_node : window_function_nodes) + { + auto & window_function_node_typed = window_function_node->as(); + + auto function_window_description = extractWindowDescriptionFromWindowNode(window_function_node_typed.getWindowNode(), planner_context); + auto window_name = function_window_description.window_name; + + auto [it, _] = window_name_to_description.emplace(window_name, std::move(function_window_description)); + auto & window_description = it->second; + + WindowFunctionDescription window_function; + window_function.function_node = nullptr; + window_function.column_name = calculateActionNodeName(window_function_node, planner_context); + window_function.aggregate_function = window_function_node_typed.getAggregateFunction(); + + const auto & parameters_nodes = window_function_node_typed.getParameters().getNodes(); + window_function.function_parameters.reserve(parameters_nodes.size()); + + for (const auto & parameter_node : parameters_nodes) + { + /// Function parameters constness validated during analysis stage + window_function.function_parameters.push_back(parameter_node->getConstantValue().getValue()); + } + + const auto & arguments_nodes = window_function_node_typed.getArguments().getNodes(); + window_function.argument_names.reserve(arguments_nodes.size()); + window_function.argument_types.reserve(arguments_nodes.size()); + + for (const auto & argument_node : arguments_nodes) + { + String argument_node_name = calculateActionNodeName(argument_node, planner_context); + window_function.argument_names.emplace_back(std::move(argument_node_name)); + window_function.argument_types.emplace_back(argument_node->getResultType()); + } + + window_description.window_functions.push_back(window_function); + } + + std::vector result; + result.reserve(window_name_to_description.size()); + + for (auto && [_, window_description] : window_name_to_description) + result.push_back(std::move(window_description)); + + return result; +} + +void sortWindowDescriptions(std::vector & window_descriptions) +{ + auto window_description_comparator = [](const WindowDescription & lhs, const WindowDescription & rhs) + { + const auto & left = lhs.full_sort_description; + const auto & right = rhs.full_sort_description; + + for (size_t i = 0; i < std::min(left.size(), right.size()); ++i) + { + if (left[i].column_name < right[i].column_name) + return true; + else if (left[i].column_name > right[i].column_name) + return false; + else if (left[i].direction < right[i].direction) + return true; + else if (left[i].direction > right[i].direction) + return false; + else if (left[i].nulls_direction < right[i].nulls_direction) + return true; + else if (left[i].nulls_direction > right[i].nulls_direction) + return false; + + assert(left[i] == right[i]); + } + + /** Note that we check the length last, because we want to put together the + * sort orders that have common prefix but different length. + */ + return left.size() > right.size(); + }; + + ::sort(window_descriptions.begin(), window_descriptions.end(), window_description_comparator); +} + +} diff --git a/src/Planner/PlannerWindowFunctions.h b/src/Planner/PlannerWindowFunctions.h new file mode 100644 index 00000000000..b31a5c1eb05 --- /dev/null +++ b/src/Planner/PlannerWindowFunctions.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +#include + +#include + +namespace DB +{ + +/// Extract and sort window description from query. +std::vector extractWindowDescriptions(const QueryTreeNodes & window_function_nodes, const PlannerContext & planner_context); + +/** Try to sort window description in such an order that the window with the longest + * sort description goes first, and all window that use its prefixes follow. + */ +void sortWindowDescriptions(std::vector & window_descriptions); + +} diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 138f24a883a..12b2b244965 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -180,4 +180,19 @@ ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & express return action_dag; } +bool sortDescriptionIsPrefix(const SortDescription & prefix, const SortDescription & full) +{ + if (prefix.size() > full.size()) + return false; + + for (size_t i = 0; i < prefix.size(); ++i) + { + if (full[i] != prefix[i]) + return false; + } + + return true; +} + + } diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 19ec8c4a76a..35913e1096f 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -44,4 +45,7 @@ ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & express const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context); +/// Returns true if prefix sort description is prefix of full sort descriptor, false otherwise +bool sortDescriptionIsPrefix(const SortDescription & prefix, const SortDescription & full); + } diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index e3eaed358f6..f5e7a438877 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -62,7 +62,7 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, } auto context = read_from_merge_tree->getContext(); - if (!context->getSettings().optimize_read_in_window_order) + if (!context->getSettings().optimize_read_in_window_order || context->getSettingsRef().use_analyzer) { return 0; } From fe84e871f870d1d1445378b80befae2787d36c88 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 15 Sep 2022 12:14:55 +0200 Subject: [PATCH 102/188] Fixed build --- src/Analyzer/CollectWindowFunctionNodes.cpp | 2 +- src/Analyzer/QueryAnalysisPass.cpp | 3 ++- src/Analyzer/SetUtils.cpp | 2 +- src/Functions/grouping.h | 4 ++-- src/Planner/PlannerAggregation.cpp | 12 ++++++++---- 5 files changed, 14 insertions(+), 9 deletions(-) diff --git a/src/Analyzer/CollectWindowFunctionNodes.cpp b/src/Analyzer/CollectWindowFunctionNodes.cpp index 25c159c8386..ba3e0bde909 100644 --- a/src/Analyzer/CollectWindowFunctionNodes.cpp +++ b/src/Analyzer/CollectWindowFunctionNodes.cpp @@ -9,7 +9,7 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_AGGREGATION; + extern const int ILLEGAL_AGGREGATION; } namespace diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 12d1d069611..9f8b754c9c5 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -3125,7 +3125,8 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc "Function GROUPING can have up to 64 arguments, but {} provided", function_arguments_size); - auto grouping_function = std::make_shared(); + bool force_grouping_standard_compatibility = context->getSettingsRef().force_grouping_standard_compatibility; + auto grouping_function = std::make_shared(force_grouping_standard_compatibility); auto grouping_function_adaptor = std::make_shared(std::move(grouping_function)); function_node.resolveAsFunction(std::move(grouping_function_adaptor), std::make_shared()); return; diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 40d088b4c05..9d33d936c39 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -83,7 +83,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & "Invalid type in set. Expected tuple, got {}", value.getTypeName()); - const auto & tuple = DB::get(value); + const auto & tuple = value.template get(); size_t tuple_size = tuple.size(); if (tuple_size != columns_size) diff --git a/src/Functions/grouping.h b/src/Functions/grouping.h index 5c4f5851635..830c509f1f5 100644 --- a/src/Functions/grouping.h +++ b/src/Functions/grouping.h @@ -79,8 +79,8 @@ public: class FunctionGrouping : public FunctionGroupingBase { public: - explicit FunctionGrouping() - : FunctionGroupingBase(ColumnNumbers()) + explicit FunctionGrouping(bool force_compatibility_) + : FunctionGroupingBase(ColumnNumbers(), force_compatibility_) {} String getName() const override { return "grouping"; } diff --git a/src/Planner/PlannerAggregation.cpp b/src/Planner/PlannerAggregation.cpp index 0d977193351..2480e47bb82 100644 --- a/src/Planner/PlannerAggregation.cpp +++ b/src/Planner/PlannerAggregation.cpp @@ -8,6 +8,8 @@ #include #include +#include + #include namespace DB @@ -99,18 +101,20 @@ public: auto grouping_set_argument_column = std::make_shared(NameAndTypePair{"__grouping_set", std::make_shared()}, column_source); function_node->getArguments().getNodes().clear(); + bool force_grouping_standard_compatibility = data.planner_context.getQueryContext()->getSettingsRef().force_grouping_standard_compatibility; + switch (data.group_by_kind) { case GroupByKind::ORDINARY: { - auto grouping_ordinary_function = std::make_shared(arguments_indexes); + auto grouping_ordinary_function = std::make_shared(arguments_indexes, force_grouping_standard_compatibility); auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_ordinary_function)); function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); break; } case GroupByKind::ROLLUP: { - auto grouping_ordinary_function = std::make_shared(arguments_indexes, aggregation_keys_size); + auto grouping_ordinary_function = std::make_shared(arguments_indexes, aggregation_keys_size, force_grouping_standard_compatibility); auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_ordinary_function)); function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); function_node->getArguments().getNodes().push_back(std::move(grouping_set_argument_column)); @@ -118,7 +122,7 @@ public: } case GroupByKind::CUBE: { - auto grouping_ordinary_function = std::make_shared(arguments_indexes, aggregation_keys_size); + auto grouping_ordinary_function = std::make_shared(arguments_indexes, aggregation_keys_size, force_grouping_standard_compatibility); auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_ordinary_function)); function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); function_node->getArguments().getNodes().push_back(std::move(grouping_set_argument_column)); @@ -126,7 +130,7 @@ public: } case GroupByKind::GROUPING_SETS: { - auto grouping_grouping_sets_function = std::make_shared(arguments_indexes, data.grouping_sets_keys_indices); + auto grouping_grouping_sets_function = std::make_shared(arguments_indexes, data.grouping_sets_keys_indices, force_grouping_standard_compatibility); auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_grouping_sets_function)); function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); function_node->getArguments().getNodes().push_back(std::move(grouping_set_argument_column)); From 2b2a70082e4f8025e088b847051b144239f69e98 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 15 Sep 2022 13:25:11 +0200 Subject: [PATCH 103/188] Rename SortColumnNode into SortNode --- src/Analyzer/IQueryTreeNode.cpp | 2 +- src/Analyzer/IQueryTreeNode.h | 2 +- src/Analyzer/QueryAnalysisPass.cpp | 39 ++++++++++--------- src/Analyzer/QueryTreeBuilder.cpp | 20 +++++----- .../{SortColumnNode.cpp => SortNode.cpp} | 22 +++++------ src/Analyzer/{SortColumnNode.h => SortNode.h} | 16 ++++---- src/Planner/Planner.cpp | 16 ++++---- src/Planner/PlannerActionsVisitor.cpp | 26 ++++++------- src/Planner/PlannerSorting.cpp | 36 ++++++++--------- 9 files changed, 90 insertions(+), 89 deletions(-) rename src/Analyzer/{SortColumnNode.cpp => SortNode.cpp} (84%) rename src/Analyzer/{SortColumnNode.h => SortNode.h} (87%) diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 71ff344deb4..9ed5ab2478e 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -30,7 +30,7 @@ const char * toString(QueryTreeNodeType type) case QueryTreeNodeType::FUNCTION: return "FUNCTION"; case QueryTreeNodeType::COLUMN: return "COLUMN"; case QueryTreeNodeType::LAMBDA: return "LAMBDA"; - case QueryTreeNodeType::SORT_COLUMN: return "SORT_COLUMN"; + case QueryTreeNodeType::SORT: return "SORT"; case QueryTreeNodeType::INTERPOLATE_COLUMN: return "INTERPOLATE_COLUMN"; case QueryTreeNodeType::WINDOW: return "WINDOW"; case QueryTreeNodeType::TABLE: return "TABLE"; diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 03de250f403..d4b2e40698c 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -37,7 +37,7 @@ enum class QueryTreeNodeType FUNCTION, COLUMN, LAMBDA, - SORT_COLUMN, + SORT, INTERPOLATE_COLUMN, WINDOW, TABLE, diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 9f8b754c9c5..accc688782b 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -40,7 +40,7 @@ #include #include #include -#include +#include #include #include #include @@ -3479,10 +3479,10 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes /// Lambda must be resolved by caller break; } - case QueryTreeNodeType::SORT_COLUMN: + case QueryTreeNodeType::SORT: { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Sort column {} is not allowed in expression. In scope {}", + "Sort {} is not allowed in expression. In scope {}", node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } @@ -3621,38 +3621,38 @@ void QueryAnalyzer::resolveSortColumnsNodeList(QueryTreeNodePtr & sort_columns_n auto & sort_columns_node_list_typed = sort_columns_node_list->as(); for (auto & node : sort_columns_node_list_typed.getNodes()) { - auto & sort_column_node = node->as(); - resolveExpressionNode(sort_column_node.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + auto & sort_node = node->as(); + resolveExpressionNode(sort_node.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - if (sort_column_node.hasFillFrom()) + if (sort_node.hasFillFrom()) { - resolveExpressionNode(sort_column_node.getFillFrom(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + resolveExpressionNode(sort_node.getFillFrom(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - const auto * constant_node = sort_column_node.getFillFrom()->as(); + const auto * constant_node = sort_node.getFillFrom()->as(); if (!constant_node || !isColumnedAsNumber(constant_node->getResultType())) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL FROM expression must be constant with numeric type. Actual {}. In scope {}", - sort_column_node.getFillFrom()->formatASTForErrorMessage(), + sort_node.getFillFrom()->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } - if (sort_column_node.hasFillTo()) + if (sort_node.hasFillTo()) { - resolveExpressionNode(sort_column_node.getFillTo(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - const auto * constant_node = sort_column_node.getFillTo()->as(); + resolveExpressionNode(sort_node.getFillTo(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + const auto * constant_node = sort_node.getFillTo()->as(); if (!constant_node || !isColumnedAsNumber(constant_node->getResultType())) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL TO expression must be constant with numeric type. Actual {}. In scope {}", - sort_column_node.getFillFrom()->formatASTForErrorMessage(), + sort_node.getFillFrom()->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } - if (sort_column_node.hasFillStep()) + if (sort_node.hasFillStep()) { - resolveExpressionNode(sort_column_node.getFillStep(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - const auto * constant_node = sort_column_node.getFillStep()->as(); + resolveExpressionNode(sort_node.getFillStep(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + const auto * constant_node = sort_node.getFillStep()->as(); if (!constant_node) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL TO expression must be constant with numeric or interval type. Actual {}. In scope {}", - sort_column_node.getFillStep()->formatASTForErrorMessage(), + sort_node.getFillStep()->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); bool is_number = isColumnedAsNumber(constant_node->getResultType()); @@ -3660,7 +3660,7 @@ void QueryAnalyzer::resolveSortColumnsNodeList(QueryTreeNodePtr & sort_columns_n if (!is_number && !is_interval) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL TO expression must be constant with numeric or interval type. Actual {}. In scope {}", - sort_column_node.getFillStep()->formatASTForErrorMessage(), + sort_node.getFillStep()->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } } @@ -4530,7 +4530,8 @@ public: { auto query_tree_node_type = node->getNodeType(); if (query_tree_node_type == QueryTreeNodeType::CONSTANT || - query_tree_node_type == QueryTreeNodeType::SORT_COLUMN) + query_tree_node_type == QueryTreeNodeType::SORT || + query_tree_node_type == QueryTreeNodeType::INTERPOLATE_COLUMN) return; auto * function_node = node->as(); diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index b88215606cc..7d3a03c115d 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -33,7 +33,7 @@ #include #include #include -#include +#include #include #include #include @@ -82,7 +82,7 @@ private: QueryTreeNodePtr buildSelectExpression(const ASTPtr & select_query, bool is_subquery, const std::string & cte_name) const; - QueryTreeNodePtr buildSortColumnList(const ASTPtr & order_by_expression_list) const; + QueryTreeNodePtr buildSortList(const ASTPtr & order_by_expression_list) const; QueryTreeNodePtr buildInterpolateColumnList(const ASTPtr & interpolate_expression_list) const; @@ -262,7 +262,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q auto select_order_by_list = select_query_typed.orderBy(); if (select_order_by_list) - current_query_tree->getOrderByNode() = buildSortColumnList(select_order_by_list); + current_query_tree->getOrderByNode() = buildSortList(select_order_by_list); auto interpolate_list = select_query_typed.interpolate(); if (interpolate_list) @@ -291,7 +291,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q return current_query_tree; } -QueryTreeNodePtr QueryTreeBuilder::buildSortColumnList(const ASTPtr & order_by_expression_list) const +QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_expression_list) const { auto list_node = std::make_shared(); @@ -313,20 +313,20 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortColumnList(const ASTPtr & order_by_e const auto & sort_expression_ast = order_by_element.children.at(0); auto sort_expression = buildExpression(sort_expression_ast); - auto sort_column_node = std::make_shared(std::move(sort_expression), + auto sort_node = std::make_shared(std::move(sort_expression), sort_direction, nulls_sort_direction, std::move(collator), order_by_element.with_fill); if (order_by_element.fill_from) - sort_column_node->getFillFrom() = buildExpression(order_by_element.fill_from); + sort_node->getFillFrom() = buildExpression(order_by_element.fill_from); if (order_by_element.fill_to) - sort_column_node->getFillTo() = buildExpression(order_by_element.fill_to); + sort_node->getFillTo() = buildExpression(order_by_element.fill_to); if (order_by_element.fill_step) - sort_column_node->getFillStep() = buildExpression(order_by_element.fill_step); + sort_node->getFillStep() = buildExpression(order_by_element.fill_step); - list_node->getNodes().push_back(std::move(sort_column_node)); + list_node->getNodes().push_back(std::move(sort_node)); } return list_node; @@ -582,7 +582,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildWindow(const ASTPtr & window_definition) window_node->getPartitionByNode() = buildExpressionList(window_definition_typed.partition_by); if (window_definition_typed.order_by) - window_node->getOrderByNode() = buildSortColumnList(window_definition_typed.order_by); + window_node->getOrderByNode() = buildSortList(window_definition_typed.order_by); if (window_definition_typed.frame_begin_offset) window_node->getFrameBeginOffsetNode() = buildExpression(window_definition_typed.frame_begin_offset); diff --git a/src/Analyzer/SortColumnNode.cpp b/src/Analyzer/SortNode.cpp similarity index 84% rename from src/Analyzer/SortColumnNode.cpp rename to src/Analyzer/SortNode.cpp index c3b9a98eb07..f9dd2054134 100644 --- a/src/Analyzer/SortColumnNode.cpp +++ b/src/Analyzer/SortNode.cpp @@ -1,4 +1,4 @@ -#include +#include #include @@ -21,7 +21,7 @@ const char * toString(SortDirection sort_direction) } } -SortColumnNode::SortColumnNode(QueryTreeNodePtr expression_, +SortNode::SortNode(QueryTreeNodePtr expression_, SortDirection sort_direction_, std::optional nulls_sort_direction_, std::shared_ptr collator_, @@ -35,7 +35,7 @@ SortColumnNode::SortColumnNode(QueryTreeNodePtr expression_, children[sort_expression_child_index] = std::move(expression_); } -String SortColumnNode::getName() const +String SortNode::getName() const { String result = getExpression()->getName(); @@ -67,9 +67,9 @@ String SortColumnNode::getName() const return result; } -void SortColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const +void SortNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { - buffer << std::string(indent, ' ') << "SORT_COLUMN id: " << format_state.getNodeId(this); + buffer << std::string(indent, ' ') << "SORT id: " << format_state.getNodeId(this); buffer << ", sort_direction: " << toString(sort_direction); if (nulls_sort_direction) @@ -102,9 +102,9 @@ void SortColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_sta } } -bool SortColumnNode::isEqualImpl(const IQueryTreeNode & rhs) const +bool SortNode::isEqualImpl(const IQueryTreeNode & rhs) const { - const auto & rhs_typed = assert_cast(rhs); + const auto & rhs_typed = assert_cast(rhs); if (sort_direction != rhs_typed.sort_direction || nulls_sort_direction != rhs_typed.nulls_sort_direction || with_fill != rhs_typed.with_fill) @@ -120,7 +120,7 @@ bool SortColumnNode::isEqualImpl(const IQueryTreeNode & rhs) const return collator->getLocale() == rhs_typed.collator->getLocale(); } -void SortColumnNode::updateTreeHashImpl(HashState & hash_state) const +void SortNode::updateTreeHashImpl(HashState & hash_state) const { hash_state.update(sort_direction); hash_state.update(nulls_sort_direction); @@ -135,7 +135,7 @@ void SortColumnNode::updateTreeHashImpl(HashState & hash_state) const } } -ASTPtr SortColumnNode::toASTImpl() const +ASTPtr SortNode::toASTImpl() const { auto result = std::make_shared(); result->direction = sort_direction == SortDirection::ASCENDING ? 1 : -1; @@ -156,9 +156,9 @@ ASTPtr SortColumnNode::toASTImpl() const return result; } -QueryTreeNodePtr SortColumnNode::cloneImpl() const +QueryTreeNodePtr SortNode::cloneImpl() const { - return std::make_shared(nullptr /*expression*/, sort_direction, nulls_sort_direction, collator, with_fill); + return std::make_shared(nullptr /*expression*/, sort_direction, nulls_sort_direction, collator, with_fill); } } diff --git a/src/Analyzer/SortColumnNode.h b/src/Analyzer/SortNode.h similarity index 87% rename from src/Analyzer/SortColumnNode.h rename to src/Analyzer/SortNode.h index 9ec73cadc4b..f003cd34e00 100644 --- a/src/Analyzer/SortColumnNode.h +++ b/src/Analyzer/SortNode.h @@ -8,12 +8,12 @@ namespace DB { -/** Sort column node represents sort column descripion that is part of ORDER BY in query tree. +/** Sort node represents sort description for expression that is part of ORDER BY in query tree. * Example: SELECT * FROM test_table ORDER BY sort_column_1, sort_column_2; - * Sort column optionally contain collation, fill from, fill to, and fill step. + * Sort node optionally contain collation, fill from, fill to, and fill step. */ -class SortColumnNode; -using SortColumnNodePtr = std::shared_ptr; +class SortNode; +using SortNodePtr = std::shared_ptr; enum class SortDirection { @@ -23,11 +23,11 @@ enum class SortDirection const char * toString(SortDirection sort_direction); -class SortColumnNode final : public IQueryTreeNode +class SortNode final : public IQueryTreeNode { public: - /// Initialize sort column node with sort expression - explicit SortColumnNode(QueryTreeNodePtr expression_, + /// Initialize sort node with sort expression + explicit SortNode(QueryTreeNodePtr expression_, SortDirection sort_direction_ = SortDirection::ASCENDING, std::optional nulls_sort_direction_ = {}, std::shared_ptr collator_ = nullptr, @@ -122,7 +122,7 @@ public: QueryTreeNodeType getNodeType() const override { - return QueryTreeNodeType::SORT_COLUMN; + return QueryTreeNodeType::SORT; } String getName() const override; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 0ad1c693339..4bd82a91f69 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -46,7 +46,7 @@ #include #include #include -#include +#include #include #include #include @@ -990,10 +990,10 @@ void Planner::buildQueryPlanIfNeeded() * WITH fill expressions must be constant nodes. */ auto & order_by_node_list = window_node.getOrderBy(); - for (auto & sort_column_node : order_by_node_list.getNodes()) + for (auto & sort_node : order_by_node_list.getNodes()) { - auto & sort_column_node_typed = sort_column_node->as(); - expression_dag_nodes = actions_visitor.visit(before_window_actions_dag, sort_column_node_typed.getExpression()); + auto & sort_node_typed = sort_node->as(); + expression_dag_nodes = actions_visitor.visit(before_window_actions_dag, sort_node_typed.getExpression()); for (auto & expression_dag_node : expression_dag_nodes) { @@ -1065,14 +1065,14 @@ void Planner::buildQueryPlanIfNeeded() std::unordered_set before_order_by_actions_dag_outputs_node_names; - /** We add only sort column sort expression in before ORDER BY actions DAG. + /** We add only sort node sort expression in before ORDER BY actions DAG. * WITH fill expressions must be constant nodes. */ auto & order_by_node_list = query_node.getOrderBy(); - for (auto & sort_column_node : order_by_node_list.getNodes()) + for (auto & sort_node : order_by_node_list.getNodes()) { - auto & sort_column_node_typed = sort_column_node->as(); - auto expression_dag_nodes = actions_visitor.visit(before_order_by_actions_dag, sort_column_node_typed.getExpression()); + auto & sort_node_typed = sort_node->as(); + auto expression_dag_nodes = actions_visitor.visit(before_order_by_actions_dag, sort_node_typed.getExpression()); for (auto & action_dag_node : expression_dag_nodes) { diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 47754587241..d7bae6d248d 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include @@ -669,32 +669,32 @@ String calculateWindowNodeActionName(const QueryTreeNodePtr & node, const Planne for (size_t i = 0; i < order_by_nodes_size; ++i) { - auto & sort_column_node = order_by_nodes[i]->as(); + auto & sort_node = order_by_nodes[i]->as(); + buffer << calculateActionNodeName(sort_node.getExpression(), planner_context, node_to_name); - buffer << calculateActionNodeName(sort_column_node.getExpression(), planner_context, node_to_name); - auto sort_direction = sort_column_node.getSortDirection(); + auto sort_direction = sort_node.getSortDirection(); buffer << (sort_direction == SortDirection::ASCENDING ? " ASC" : " DESC"); - auto nulls_sort_direction = sort_column_node.getNullsSortDirection(); + auto nulls_sort_direction = sort_node.getNullsSortDirection(); if (nulls_sort_direction) buffer << " NULLS " << (nulls_sort_direction == sort_direction ? "LAST" : "FIRST"); - if (auto collator = sort_column_node.getCollator()) + if (auto collator = sort_node.getCollator()) buffer << " COLLATE " << collator->getLocale(); - if (sort_column_node.withFill()) + if (sort_node.withFill()) { buffer << " WITH FILL"; - if (sort_column_node.hasFillFrom()) - buffer << " FROM " << calculateActionNodeName(sort_column_node.getFillFrom(), planner_context, node_to_name); + if (sort_node.hasFillFrom()) + buffer << " FROM " << calculateActionNodeName(sort_node.getFillFrom(), planner_context, node_to_name); - if (sort_column_node.hasFillTo()) - buffer << " TO " << calculateActionNodeName(sort_column_node.getFillTo(), planner_context, node_to_name); + if (sort_node.hasFillTo()) + buffer << " TO " << calculateActionNodeName(sort_node.getFillTo(), planner_context, node_to_name); - if (sort_column_node.hasFillStep()) - buffer << " STEP " << calculateActionNodeName(sort_column_node.getFillStep(), planner_context, node_to_name); + if (sort_node.hasFillStep()) + buffer << " STEP " << calculateActionNodeName(sort_node.getFillStep(), planner_context, node_to_name); } if (i + 1 != order_by_nodes_size) diff --git a/src/Planner/PlannerSorting.cpp b/src/Planner/PlannerSorting.cpp index 17304fa7766..2ce497b7bbd 100644 --- a/src/Planner/PlannerSorting.cpp +++ b/src/Planner/PlannerSorting.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include #include @@ -50,40 +50,40 @@ std::pair> extractWithFillStepValue(const Que return {constant_value.getValue(), {}}; } -FillColumnDescription extractWithFillDescription(const SortColumnNode & sort_column_node) +FillColumnDescription extractWithFillDescription(const SortNode & sort_node) { FillColumnDescription fill_column_description; - if (sort_column_node.hasFillFrom()) + if (sort_node.hasFillFrom()) { - auto extract_result = extractWithFillValue(sort_column_node.getFillFrom()); + auto extract_result = extractWithFillValue(sort_node.getFillFrom()); fill_column_description.fill_from = std::move(extract_result.first); fill_column_description.fill_from_type = std::move(extract_result.second); } - if (sort_column_node.hasFillTo()) + if (sort_node.hasFillTo()) { - auto extract_result = extractWithFillValue(sort_column_node.getFillTo()); + auto extract_result = extractWithFillValue(sort_node.getFillTo()); fill_column_description.fill_to = std::move(extract_result.first); fill_column_description.fill_to_type = std::move(extract_result.second); } - if (sort_column_node.hasFillStep()) + if (sort_node.hasFillStep()) { - auto extract_result = extractWithFillStepValue(sort_column_node.getFillStep()); + auto extract_result = extractWithFillStepValue(sort_node.getFillStep()); fill_column_description.fill_step = std::move(extract_result.first); fill_column_description.step_kind = std::move(extract_result.second); } else { - fill_column_description.fill_step = Field(sort_column_node.getSortDirection() == SortDirection::ASCENDING ? 1 : -1); + fill_column_description.fill_step = Field(sort_node.getSortDirection() == SortDirection::ASCENDING ? 1 : -1); } if (applyVisitor(FieldVisitorAccurateEquals(), fill_column_description.fill_step, Field{0})) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL STEP value cannot be zero"); - if (sort_column_node.getSortDirection() == SortDirection::ASCENDING) + if (sort_node.getSortDirection() == SortDirection::ASCENDING) { if (applyVisitor(FieldVisitorAccurateLess(), fill_column_description.fill_step, Field{0})) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, @@ -122,22 +122,22 @@ SortDescription extractSortDescription(const QueryTreeNodePtr & order_by_node, c SortDescription sort_column_description; sort_column_description.reserve(order_by_list_node.getNodes().size()); - for (const auto & sort_column_node : order_by_list_node.getNodes()) + for (const auto & sort_node : order_by_list_node.getNodes()) { - auto & sort_column_node_typed = sort_column_node->as(); + auto & sort_node_typed = sort_node->as(); - auto column_name = calculateActionNodeName(sort_column_node_typed.getExpression(), planner_context); - std::shared_ptr collator = sort_column_node_typed.getCollator(); - int direction = sort_column_node_typed.getSortDirection() == SortDirection::ASCENDING ? 1 : -1; + auto column_name = calculateActionNodeName(sort_node_typed.getExpression(), planner_context); + std::shared_ptr collator = sort_node_typed.getCollator(); + int direction = sort_node_typed.getSortDirection() == SortDirection::ASCENDING ? 1 : -1; int nulls_direction = direction; - auto nulls_sort_direction = sort_column_node_typed.getNullsSortDirection(); + auto nulls_sort_direction = sort_node_typed.getNullsSortDirection(); if (nulls_sort_direction) nulls_direction = *nulls_sort_direction == SortDirection::ASCENDING ? 1 : -1; - if (sort_column_node_typed.withFill()) + if (sort_node_typed.withFill()) { - FillColumnDescription fill_description = extractWithFillDescription(sort_column_node_typed); + FillColumnDescription fill_description = extractWithFillDescription(sort_node_typed); sort_column_description.emplace_back(column_name, direction, nulls_direction, collator, true /*with_fill*/, fill_description); } else From c7d51ca83dda7721dbf38eb91bbe04ffcb3cf2e7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 15 Sep 2022 13:45:01 +0200 Subject: [PATCH 104/188] Rename InterpolateColumnNode into InterpolateNode --- src/Analyzer/IQueryTreeNode.cpp | 2 +- src/Analyzer/IQueryTreeNode.h | 2 +- ...lateColumnNode.cpp => InterpolateNode.cpp} | 18 +++++------ ...erpolateColumnNode.h => InterpolateNode.h} | 14 ++++---- src/Analyzer/QueryAnalysisPass.cpp | 32 ++++++++++--------- src/Analyzer/QueryTreeBuilder.cpp | 6 ++-- src/Planner/Planner.cpp | 17 +++++----- 7 files changed, 47 insertions(+), 44 deletions(-) rename src/Analyzer/{InterpolateColumnNode.cpp => InterpolateNode.cpp} (65%) rename src/Analyzer/{InterpolateColumnNode.h => InterpolateNode.h} (73%) diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 9ed5ab2478e..2f555b29708 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -31,7 +31,7 @@ const char * toString(QueryTreeNodeType type) case QueryTreeNodeType::COLUMN: return "COLUMN"; case QueryTreeNodeType::LAMBDA: return "LAMBDA"; case QueryTreeNodeType::SORT: return "SORT"; - case QueryTreeNodeType::INTERPOLATE_COLUMN: return "INTERPOLATE_COLUMN"; + case QueryTreeNodeType::INTERPOLATE: return "INTERPOLATE"; case QueryTreeNodeType::WINDOW: return "WINDOW"; case QueryTreeNodeType::TABLE: return "TABLE"; case QueryTreeNodeType::TABLE_FUNCTION: return "TABLE_FUNCTION"; diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index d4b2e40698c..22b3b658f3e 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -38,7 +38,7 @@ enum class QueryTreeNodeType COLUMN, LAMBDA, SORT, - INTERPOLATE_COLUMN, + INTERPOLATE, WINDOW, TABLE, TABLE_FUNCTION, diff --git a/src/Analyzer/InterpolateColumnNode.cpp b/src/Analyzer/InterpolateNode.cpp similarity index 65% rename from src/Analyzer/InterpolateColumnNode.cpp rename to src/Analyzer/InterpolateNode.cpp index 33e1821e9c1..628caaaf219 100644 --- a/src/Analyzer/InterpolateColumnNode.cpp +++ b/src/Analyzer/InterpolateNode.cpp @@ -1,4 +1,4 @@ -#include +#include #include @@ -10,14 +10,14 @@ namespace DB { -InterpolateColumnNode::InterpolateColumnNode(QueryTreeNodePtr expression_, QueryTreeNodePtr interpolate_expression_) +InterpolateNode::InterpolateNode(QueryTreeNodePtr expression_, QueryTreeNodePtr interpolate_expression_) { children.resize(children_size); children[expression_child_index] = std::move(expression_); children[interpolate_expression_child_index] = std::move(interpolate_expression_); } -String InterpolateColumnNode::getName() const +String InterpolateNode::getName() const { String result = getExpression()->getName(); result += " AS "; @@ -26,7 +26,7 @@ String InterpolateColumnNode::getName() const return result; } -void InterpolateColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const +void InterpolateNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "INTERPOLATE_COLUMN id: " << format_state.getNodeId(this); @@ -37,18 +37,18 @@ void InterpolateColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & for getInterpolateExpression()->dumpTreeImpl(buffer, format_state, indent + 4); } -bool InterpolateColumnNode::isEqualImpl(const IQueryTreeNode &) const +bool InterpolateNode::isEqualImpl(const IQueryTreeNode &) const { /// No state in interpolate column node return true; } -void InterpolateColumnNode::updateTreeHashImpl(HashState &) const +void InterpolateNode::updateTreeHashImpl(HashState &) const { /// No state in interpolate column node } -ASTPtr InterpolateColumnNode::toASTImpl() const +ASTPtr InterpolateNode::toASTImpl() const { auto result = std::make_shared(); result->column = getExpression()->toAST()->getColumnName(); @@ -58,9 +58,9 @@ ASTPtr InterpolateColumnNode::toASTImpl() const return result; } -QueryTreeNodePtr InterpolateColumnNode::cloneImpl() const +QueryTreeNodePtr InterpolateNode::cloneImpl() const { - return std::make_shared(nullptr /*expression*/, nullptr /*interpolate_expression*/); + return std::make_shared(nullptr /*expression*/, nullptr /*interpolate_expression*/); } } diff --git a/src/Analyzer/InterpolateColumnNode.h b/src/Analyzer/InterpolateNode.h similarity index 73% rename from src/Analyzer/InterpolateColumnNode.h rename to src/Analyzer/InterpolateNode.h index 1ff66710898..d9efffdc79f 100644 --- a/src/Analyzer/InterpolateColumnNode.h +++ b/src/Analyzer/InterpolateNode.h @@ -6,19 +6,19 @@ namespace DB { -/** Interpolate column node represents single column interpolation in INTERPOLATE section that part of ORDER BY in query tree. +/** Interpolate node represents expression interpolation in INTERPOLATE section that is part of ORDER BY section in query tree. * Example: SELECT * FROM test_table ORDER BY id WITH FILL INTERPOLATE value AS value + 1; * value - expression to interpolate. * value + 1 - interpolate expression. */ -class InterpolateColumnNode; -using InterpolateColumnNodePtr = std::shared_ptr; +class InterpolateNode; +using InterpolateNodePtr = std::shared_ptr; -class InterpolateColumnNode final : public IQueryTreeNode +class InterpolateNode final : public IQueryTreeNode { public: - /// Initialize interpolate column node with expression and interpolate expression - explicit InterpolateColumnNode(QueryTreeNodePtr expression_, QueryTreeNodePtr interpolate_expression_); + /// Initialize interpolate node with expression and interpolate expression + explicit InterpolateNode(QueryTreeNodePtr expression_, QueryTreeNodePtr interpolate_expression_); /// Get expression const QueryTreeNodePtr & getExpression() const @@ -46,7 +46,7 @@ public: QueryTreeNodeType getNodeType() const override { - return QueryTreeNodeType::INTERPOLATE_COLUMN; + return QueryTreeNodeType::INTERPOLATE; } String getName() const override; diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index accc688782b..a6b3f159aca 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -41,7 +41,7 @@ #include #include #include -#include +#include #include #include #include @@ -933,9 +933,9 @@ private: void resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); - void resolveSortColumnsNodeList(QueryTreeNodePtr & sort_columns_node_list, IdentifierResolveScope & scope); + void resolveSortColumnsNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope); - void resolveInterpolateColumnsNodeList(QueryTreeNodePtr & sort_columns_node_list, IdentifierResolveScope & scope); + void resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpolate_node_list, IdentifierResolveScope & scope); void resolveWindowNodeList(QueryTreeNodePtr & window_node_list, IdentifierResolveScope & scope); @@ -3486,7 +3486,7 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } - case QueryTreeNodeType::INTERPOLATE_COLUMN: + case QueryTreeNodeType::INTERPOLATE: { throw Exception(ErrorCodes::LOGICAL_ERROR, "Interpolate column {} is not allowed in expression. In scope {}", @@ -3616,10 +3616,11 @@ void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, Iden /** Resolve sort columns nodes list. */ -void QueryAnalyzer::resolveSortColumnsNodeList(QueryTreeNodePtr & sort_columns_node_list, IdentifierResolveScope & scope) +void QueryAnalyzer::resolveSortColumnsNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope) { - auto & sort_columns_node_list_typed = sort_columns_node_list->as(); - for (auto & node : sort_columns_node_list_typed.getNodes()) + auto & sort_node_list_typed = sort_node_list->as(); + + for (auto & node : sort_node_list_typed.getNodes()) { auto & sort_node = node->as(); resolveExpressionNode(sort_node.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); @@ -3668,15 +3669,16 @@ void QueryAnalyzer::resolveSortColumnsNodeList(QueryTreeNodePtr & sort_columns_n /** Resolve interpolate columns nodes list. */ -void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & sort_columns_node_list, IdentifierResolveScope & scope) +void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpolate_node_list, IdentifierResolveScope & scope) { - auto & sort_columns_node_list_typed = sort_columns_node_list->as(); - for (auto & node : sort_columns_node_list_typed.getNodes()) - { - auto & interpolate_column_node = node->as(); + auto & interpolate_node_list_typed = interpolate_node_list->as(); - resolveExpressionNode(interpolate_column_node.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - resolveExpressionNode(interpolate_column_node.getInterpolateExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + for (auto & interpolate_node : interpolate_node_list_typed.getNodes()) + { + auto & interpolate_node_typed = interpolate_node->as(); + + resolveExpressionNode(interpolate_node_typed.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + resolveExpressionNode(interpolate_node_typed.getInterpolateExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } } @@ -4531,7 +4533,7 @@ public: auto query_tree_node_type = node->getNodeType(); if (query_tree_node_type == QueryTreeNodeType::CONSTANT || query_tree_node_type == QueryTreeNodeType::SORT || - query_tree_node_type == QueryTreeNodeType::INTERPOLATE_COLUMN) + query_tree_node_type == QueryTreeNodeType::INTERPOLATE) return; auto * function_node = node->as(); diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 7d3a03c115d..489afafdf9f 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -34,7 +34,7 @@ #include #include #include -#include +#include #include #include #include @@ -344,9 +344,9 @@ QueryTreeNodePtr QueryTreeBuilder::buildInterpolateColumnList(const ASTPtr & int const auto & interpolate_element = expression->as(); auto expression_to_interpolate = std::make_shared(Identifier(interpolate_element.column)); auto interpolate_expression = buildExpression(interpolate_element.expr); - auto interpolate_column_node = std::make_shared(std::move(expression_to_interpolate), std::move(interpolate_expression)); + auto interpolate_node = std::make_shared(std::move(expression_to_interpolate), std::move(interpolate_expression)); - list_node->getNodes().push_back(std::move(interpolate_column_node)); + list_node->getNodes().push_back(std::move(interpolate_node)); } return list_node; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 4bd82a91f69..171fc822fb9 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -47,7 +47,7 @@ #include #include #include -#include +#include #include #include #include @@ -1414,9 +1414,10 @@ void Planner::buildQueryPlanIfNeeded() { auto interpolate_actions_dag = std::make_shared(); - auto & interpolate_column_list_node = query_node.getInterpolate()->as(); - auto & interpolate_column_list_nodes = interpolate_column_list_node.getNodes(); - if (interpolate_column_list_nodes.empty()) + auto & interpolate_list_node = query_node.getInterpolate()->as(); + auto & interpolate_list_nodes = interpolate_list_node.getNodes(); + + if (interpolate_list_nodes.empty()) { auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); for (auto & query_plan_column : query_plan_columns) @@ -1430,11 +1431,11 @@ void Planner::buildQueryPlanIfNeeded() } else { - for (auto & interpolate_column_node : interpolate_column_list_node.getNodes()) + for (auto & interpolate_node : interpolate_list_nodes) { - auto & interpolate_column_node_typed = interpolate_column_node->as(); - auto expression_to_interpolate_expression_nodes = actions_visitor.visit(interpolate_actions_dag, interpolate_column_node_typed.getExpression()); - auto interpolate_expression_nodes = actions_visitor.visit(interpolate_actions_dag, interpolate_column_node_typed.getInterpolateExpression()); + auto & interpolate_node_typed = interpolate_node->as(); + auto expression_to_interpolate_expression_nodes = actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getExpression()); + auto interpolate_expression_nodes = actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); if (expression_to_interpolate_expression_nodes.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression to interpolate expected to have single action node"); From 8a39c3e1d5ffed34a1ffbd0906dc74774b567ec0 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 15 Sep 2022 16:09:30 +0200 Subject: [PATCH 105/188] Separate planner JOIN TREE processing --- src/Planner/Planner.cpp | 486 +----------------------------- src/Planner/PlannerJoinTree.cpp | 511 ++++++++++++++++++++++++++++++++ src/Planner/PlannerJoinTree.h | 20 ++ 3 files changed, 532 insertions(+), 485 deletions(-) create mode 100644 src/Planner/PlannerJoinTree.cpp create mode 100644 src/Planner/PlannerJoinTree.h diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 171fc822fb9..61d97daf8bc 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -7,14 +7,10 @@ #include #include -#include #include -#include #include #include -#include #include -#include #include #include #include @@ -33,10 +29,6 @@ #include #include -#include -#include -#include -#include #include #include @@ -44,16 +36,12 @@ #include #include #include -#include -#include #include #include #include #include #include #include -#include -#include #include #include #include @@ -71,6 +59,7 @@ #include #include #include +#include namespace DB { @@ -105,479 +94,6 @@ namespace ErrorCodes namespace { -QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, - SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, - PlannerContextPtr & planner_context); - -QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, - SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, - PlannerContextPtr & planner_context) -{ - auto * table_node = table_expression->as(); - auto * table_function_node = table_expression->as(); - auto * query_node = table_expression->as(); - auto * union_node = table_expression->as(); - - QueryPlan query_plan; - - auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); - - if (table_node || table_function_node) - { - const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); - const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); - - auto table_expression_query_info = select_query_info; - table_expression_query_info.table_expression = table_expression; - - if (table_node) - table_expression_query_info.table_expression_modifiers = table_node->getTableExpressionModifiers(); - else - table_expression_query_info.table_expression_modifiers = table_function_node->getTableExpressionModifiers(); - - auto from_stage = storage->getQueryProcessingStage(planner_context->getQueryContext(), select_query_options.to_stage, storage_snapshot, table_expression_query_info); - const auto & columns_names = table_expression_data.getColumnsNames(); - Names column_names(columns_names.begin(), columns_names.end()); - - std::optional read_additional_column; - - if (column_names.empty()) - { - auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); - read_additional_column = column_names_and_types.front(); - } - - if (read_additional_column) - { - const auto & column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(*read_additional_column, table_expression); - column_names.push_back(read_additional_column->name); - table_expression_data.addColumn(*read_additional_column, column_identifier); - } - - if (!column_names.empty()) - { - const auto & query_context = planner_context->getQueryContext(); - size_t max_block_size = query_context->getSettingsRef().max_block_size; - size_t max_streams = query_context->getSettingsRef().max_threads; - storage->read(query_plan, column_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); - } - - /// Create step which reads from empty source if storage has no data. - if (!query_plan.isInitialized()) - { - auto source_header = storage_snapshot->getSampleBlockForColumns(column_names); - Pipe pipe(std::make_shared(source_header)); - auto read_from_pipe = std::make_unique(std::move(pipe)); - read_from_pipe->setStepDescription("Read from NullSource"); - query_plan.addStep(std::move(read_from_pipe)); - } - } - else if (query_node || union_node) - { - auto subquery_options = select_query_options.subquery(); - auto subquery_context = buildSubqueryContext(planner_context->getQueryContext()); - Planner subquery_planner(table_expression, subquery_options, std::move(subquery_context), planner_context->getGlobalPlannerContext()); - subquery_planner.buildQueryPlanIfNeeded(); - query_plan = std::move(subquery_planner).extractQueryPlan(); - } - else - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected table, table function, query or union. Actual {}", table_expression->formatASTForErrorMessage()); - } - - auto rename_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); - - for (const auto & [column_name, column_identifier] : table_expression_data.getColumnNameToIdentifier()) - { - auto position = query_plan.getCurrentDataStream().header.getPositionByName(column_name); - const auto * node_to_rename = rename_actions_dag->getOutputs()[position]; - rename_actions_dag->getOutputs()[position] = &rename_actions_dag->addAlias(*node_to_rename, column_identifier); - } - - auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), rename_actions_dag); - rename_step->setStepDescription("Change column names to column identifiers"); - query_plan.addStep(std::move(rename_step)); - - return query_plan; -} - -QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, - SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, - PlannerContextPtr & planner_context) -{ - auto & join_node = join_tree_node->as(); - - auto left_plan = buildQueryPlanForJoinTreeNode(join_node.getLeftTableExpression(), - select_query_info, - select_query_options, - planner_context); - auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - - auto right_plan = buildQueryPlanForJoinTreeNode(join_node.getRightTableExpression(), - select_query_info, - select_query_options, - planner_context); - auto right_plan_output_columns = right_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - - if (join_node.getStrictness() == JoinStrictness::Asof) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "JOIN {} ASOF is not supported", - join_node.formatASTForErrorMessage()); - - JoinClausesAndActions join_clauses_and_actions; - - if (join_node.isOnJoinExpression()) - { - join_clauses_and_actions = buildJoinClausesAndActions(left_plan_output_columns, - right_plan_output_columns, - join_tree_node, - planner_context); - - join_clauses_and_actions.left_join_expressions_actions->projectInput(); - auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); - left_join_expressions_actions_step->setStepDescription("JOIN actions"); - left_plan.addStep(std::move(left_join_expressions_actions_step)); - - join_clauses_and_actions.right_join_expressions_actions->projectInput(); - auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); - right_join_expressions_actions_step->setStepDescription("JOIN actions"); - right_plan.addStep(std::move(right_join_expressions_actions_step)); - } - - std::unordered_map left_plan_column_name_to_cast_type; - std::unordered_map right_plan_column_name_to_cast_type; - - if (join_node.isUsingJoinExpression()) - { - auto & join_node_using_columns_list = join_node.getJoinExpression()->as(); - for (auto & join_node_using_node : join_node_using_columns_list.getNodes()) - { - auto & join_node_using_column_node = join_node_using_node->as(); - auto & inner_columns_list = join_node_using_column_node.getExpressionOrThrow()->as(); - - auto & left_inner_column_node = inner_columns_list.getNodes().at(0); - auto & left_inner_column = left_inner_column_node->as(); - - auto & right_inner_column_node = inner_columns_list.getNodes().at(1); - auto & right_inner_column = right_inner_column_node->as(); - - const auto & join_node_using_column_node_type = join_node_using_column_node.getColumnType(); - if (!left_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) - { - auto left_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(left_inner_column_node); - left_plan_column_name_to_cast_type.emplace(left_inner_column_identifier, join_node_using_column_node_type); - } - - if (!right_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) - { - auto right_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(right_inner_column_node); - right_plan_column_name_to_cast_type.emplace(right_inner_column_identifier, join_node_using_column_node_type); - } - } - } - - auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map & plan_column_name_to_cast_type) - { - auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); - - for (auto & output_node : cast_actions_dag->getOutputs()) - { - auto it = plan_column_name_to_cast_type.find(output_node->result_name); - if (it == plan_column_name_to_cast_type.end()) - continue; - - const auto & cast_type = it->second; - auto cast_type_name = cast_type->getName(); - Field cast_type_constant_value(cast_type_name); - - ColumnWithTypeAndName column; - column.name = calculateConstantActionNodeName(cast_type_constant_value); - column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); - column.type = std::make_shared(); - - const auto * cast_type_constant_node = &cast_actions_dag->addColumn(std::move(column)); - - FunctionCastBase::Diagnostic diagnostic = {output_node->result_name, output_node->result_name}; - FunctionOverloadResolverPtr func_builder_cast - = CastInternalOverloadResolver::createImpl(std::move(diagnostic)); - - ActionsDAG::NodeRawConstPtrs children = {output_node, cast_type_constant_node}; - output_node = &cast_actions_dag->addFunction(func_builder_cast, std::move(children), output_node->result_name); - } - - cast_actions_dag->projectInput(); - auto cast_join_columns_step - = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); - cast_join_columns_step->setStepDescription("Cast JOIN USING columns"); - plan_to_add_cast.addStep(std::move(cast_join_columns_step)); - }; - - if (!left_plan_column_name_to_cast_type.empty()) - join_cast_plan_output_nodes(left_plan, left_plan_column_name_to_cast_type); - - if (!right_plan_column_name_to_cast_type.empty()) - join_cast_plan_output_nodes(right_plan, right_plan_column_name_to_cast_type); - - const auto & query_context = planner_context->getQueryContext(); - JoinKind join_kind = join_node.getKind(); - bool join_use_nulls = query_context->getSettingsRef().join_use_nulls; - auto to_nullable_function = FunctionFactory::instance().get("toNullable", query_context); - - auto join_cast_plan_columns_to_nullable = [&](QueryPlan & plan_to_add_cast) - { - auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); - - for (auto & output_node : cast_actions_dag->getOutputs()) - { - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name)) - output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); - } - - cast_actions_dag->projectInput(); - auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); - cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable"); - plan_to_add_cast.addStep(std::move(cast_join_columns_step)); - }; - - if (join_use_nulls) - { - if (isFull(join_kind)) - { - join_cast_plan_columns_to_nullable(left_plan); - join_cast_plan_columns_to_nullable(right_plan); - } - else if (isLeft(join_kind)) - { - join_cast_plan_columns_to_nullable(right_plan); - } - else if (isRight(join_kind)) - { - join_cast_plan_columns_to_nullable(left_plan); - } - } - - auto table_join = std::make_shared(); - table_join->getTableJoin() = join_node.toASTTableJoin()->as(); - if (join_node.getKind() == JoinKind::Comma) - table_join->getTableJoin().kind = JoinKind::Cross; - table_join->getTableJoin().strictness = JoinStrictness::All; - - if (join_node.isOnJoinExpression()) - { - const auto & join_clauses = join_clauses_and_actions.join_clauses; - auto & table_join_clauses = table_join->getClauses(); - - for (const auto & join_clause : join_clauses) - { - table_join_clauses.emplace_back(); - auto & table_join_clause = table_join_clauses.back(); - - const auto & join_clause_left_key_nodes = join_clause.getLeftKeyNodes(); - const auto & join_clause_right_key_nodes = join_clause.getRightKeyNodes(); - - size_t join_clause_key_nodes_size = join_clause_left_key_nodes.size(); - assert(join_clause_key_nodes_size == join_clause_right_key_nodes.size()); - - for (size_t i = 0; i < join_clause_key_nodes_size; ++i) - { - table_join_clause.key_names_left.push_back(join_clause_left_key_nodes[i]->result_name); - table_join_clause.key_names_right.push_back(join_clause_right_key_nodes[i]->result_name); - } - - const auto & join_clause_get_left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); - if (!join_clause_get_left_filter_condition_nodes.empty()) - { - if (join_clause_get_left_filter_condition_nodes.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "JOIN {} left filter conditions size must be 1. Actual {}", - join_node.formatASTForErrorMessage(), - join_clause_get_left_filter_condition_nodes.size()); - - const auto & join_clause_left_filter_condition_name = join_clause_get_left_filter_condition_nodes[0]->result_name; - table_join_clause.analyzer_left_filter_condition_column_name = join_clause_left_filter_condition_name; - } - - const auto & join_clause_get_right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); - if (!join_clause_get_right_filter_condition_nodes.empty()) - { - if (join_clause_get_right_filter_condition_nodes.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "JOIN {} right filter conditions size must be 1. Actual {}", - join_node.formatASTForErrorMessage(), - join_clause_get_right_filter_condition_nodes.size()); - - const auto & join_clause_right_filter_condition_name = join_clause_get_right_filter_condition_nodes[0]->result_name; - table_join_clause.analyzer_right_filter_condition_column_name = join_clause_right_filter_condition_name; - } - } - } - else if (join_node.isUsingJoinExpression()) - { - auto & table_join_clauses = table_join->getClauses(); - table_join_clauses.emplace_back(); - auto & table_join_clause = table_join_clauses.back(); - - auto & using_list = join_node.getJoinExpression()->as(); - - for (auto & join_using_node : using_list.getNodes()) - { - auto & join_using_column_node = join_using_node->as(); - if (!join_using_column_node.getExpression() || - join_using_column_node.getExpression()->getNodeType() != QueryTreeNodeType::LIST) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "JOIN {} column in USING does not have inner columns", - join_node.formatASTForErrorMessage()); - - auto & using_join_columns_list = join_using_column_node.getExpression()->as(); - auto & using_join_left_join_column_node = using_join_columns_list.getNodes().at(0); - auto & using_join_right_join_column_node = using_join_columns_list.getNodes().at(1); - - auto left_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_left_join_column_node); - auto right_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_right_join_column_node); - - table_join_clause.key_names_left.push_back(left_column_identifier); - table_join_clause.key_names_right.push_back(right_column_identifier); - } - } - - auto left_table_names = left_plan.getCurrentDataStream().header.getNames(); - NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); - - auto columns_from_joined_table = right_plan.getCurrentDataStream().header.getNamesAndTypesList(); - table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); - - for (auto & column_from_joined_table : columns_from_joined_table) - { - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name)) - table_join->addJoinedColumn(column_from_joined_table); - } - - size_t max_block_size = query_context->getSettingsRef().max_block_size; - size_t max_streams = query_context->getSettingsRef().max_threads; - - JoinPtr join_ptr = std::make_shared(table_join, right_plan.getCurrentDataStream().header, false /*any_take_last_row*/); - QueryPlanStepPtr join_step = std::make_unique( - left_plan.getCurrentDataStream(), - right_plan.getCurrentDataStream(), - join_ptr, - max_block_size, - max_streams, - false /*optimize_read_in_order*/); - - join_step->setStepDescription(fmt::format("JOIN {}", JoinPipelineType::FillRightFirst)); - - std::vector plans; - plans.emplace_back(std::make_unique(std::move(left_plan))); - plans.emplace_back(std::make_unique(std::move(right_plan))); - - auto result_plan = QueryPlan(); - result_plan.unitePlans(std::move(join_step), {std::move(plans)}); - - auto drop_unused_columns_after_join_actions_dag = std::make_shared(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); - ActionsDAG::NodeRawConstPtrs updated_outputs; - std::unordered_set updated_outputs_names; - - for (auto & output : drop_unused_columns_after_join_actions_dag->getOutputs()) - { - if (updated_outputs_names.contains(output->result_name) || !planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output->result_name)) - continue; - - updated_outputs.push_back(output); - updated_outputs_names.insert(output->result_name); - } - - drop_unused_columns_after_join_actions_dag->getOutputs() = std::move(updated_outputs); - - auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentDataStream(), std::move(drop_unused_columns_after_join_actions_dag)); - drop_unused_columns_after_join_transform_step->setStepDescription("DROP unused columns after JOIN"); - result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); - - return result_plan; -} - -QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression, - SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, - PlannerContextPtr & planner_context) -{ - auto & array_join_node = table_expression->as(); - - auto left_plan = buildQueryPlanForJoinTreeNode(array_join_node.getTableExpression(), - select_query_info, - select_query_options, - planner_context); - auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - - ActionsDAGPtr array_join_action_dag = std::make_shared(left_plan_output_columns); - PlannerActionsVisitor actions_visitor(planner_context); - - NameSet array_join_columns; - for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) - { - auto & array_join_expression_column = array_join_expression->as(); - const auto & array_join_column_name = array_join_expression_column.getColumnName(); - array_join_columns.insert(array_join_column_name); - - auto expression_dag_index_nodes = actions_visitor.visit(array_join_action_dag, array_join_expression_column.getExpressionOrThrow()); - for (auto & expression_dag_index_node : expression_dag_index_nodes) - { - const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_name); - array_join_action_dag->getOutputs().push_back(array_join_column_node); - } - } - - array_join_action_dag->projectInput(); - auto array_join_actions = std::make_unique(left_plan.getCurrentDataStream(), array_join_action_dag); - array_join_actions->setStepDescription("ARRAY JOIN actions"); - left_plan.addStep(std::move(array_join_actions)); - - auto array_join_action = std::make_shared(array_join_columns, array_join_node.isLeft(), planner_context->getQueryContext()); - auto array_join_step = std::make_unique(left_plan.getCurrentDataStream(), std::move(array_join_action)); - array_join_step->setStepDescription("ARRAY JOIN"); - left_plan.addStep(std::move(array_join_step)); - - return left_plan; -} - -QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, - SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, - PlannerContextPtr & planner_context) -{ - auto join_tree_node_type = join_tree_node->getNodeType(); - - switch (join_tree_node_type) - { - case QueryTreeNodeType::QUERY: - [[fallthrough]]; - case QueryTreeNodeType::UNION: - [[fallthrough]]; - case QueryTreeNodeType::TABLE: - [[fallthrough]]; - case QueryTreeNodeType::TABLE_FUNCTION: - { - return buildQueryPlanForTableExpression(join_tree_node, select_query_info, select_query_options, planner_context); - } - case QueryTreeNodeType::JOIN: - { - return buildQueryPlanForJoinNode(join_tree_node, select_query_info, select_query_options, planner_context); - } - case QueryTreeNodeType::ARRAY_JOIN: - { - return buildQueryPlanForArrayJoinNode(join_tree_node, select_query_info, select_query_options, planner_context); - } - default: - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected query, table, table function, join or array join query node. Actual {}", - join_tree_node->formatASTForErrorMessage()); - } - } -} - void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, const SelectQueryOptions & select_query_options, const PlannerContextPtr & planner_context) { if (select_query_options.is_subquery) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp new file mode 100644 index 00000000000..d5110f1a1b1 --- /dev/null +++ b/src/Planner/PlannerJoinTree.cpp @@ -0,0 +1,511 @@ +#include + +#include + +#include +#include + +#include + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContextPtr & planner_context) +{ + auto * table_node = table_expression->as(); + auto * table_function_node = table_expression->as(); + auto * query_node = table_expression->as(); + auto * union_node = table_expression->as(); + + QueryPlan query_plan; + + auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); + + if (table_node || table_function_node) + { + const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + + auto table_expression_query_info = select_query_info; + table_expression_query_info.table_expression = table_expression; + + if (table_node) + table_expression_query_info.table_expression_modifiers = table_node->getTableExpressionModifiers(); + else + table_expression_query_info.table_expression_modifiers = table_function_node->getTableExpressionModifiers(); + + auto from_stage = storage->getQueryProcessingStage(planner_context->getQueryContext(), select_query_options.to_stage, storage_snapshot, table_expression_query_info); + const auto & columns_names = table_expression_data.getColumnsNames(); + Names column_names(columns_names.begin(), columns_names.end()); + + std::optional read_additional_column; + + if (column_names.empty()) + { + auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); + read_additional_column = column_names_and_types.front(); + } + + if (read_additional_column) + { + const auto & column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(*read_additional_column, table_expression); + column_names.push_back(read_additional_column->name); + table_expression_data.addColumn(*read_additional_column, column_identifier); + } + + if (!column_names.empty()) + { + const auto & query_context = planner_context->getQueryContext(); + size_t max_block_size = query_context->getSettingsRef().max_block_size; + size_t max_streams = query_context->getSettingsRef().max_threads; + storage->read(query_plan, column_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); + } + + /// Create step which reads from empty source if storage has no data. + if (!query_plan.isInitialized()) + { + auto source_header = storage_snapshot->getSampleBlockForColumns(column_names); + Pipe pipe(std::make_shared(source_header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource"); + query_plan.addStep(std::move(read_from_pipe)); + } + } + else if (query_node || union_node) + { + auto subquery_options = select_query_options.subquery(); + auto subquery_context = buildSubqueryContext(planner_context->getQueryContext()); + Planner subquery_planner(table_expression, subquery_options, std::move(subquery_context), planner_context->getGlobalPlannerContext()); + subquery_planner.buildQueryPlanIfNeeded(); + query_plan = std::move(subquery_planner).extractQueryPlan(); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected table, table function, query or union. Actual {}", table_expression->formatASTForErrorMessage()); + } + + auto rename_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + + for (const auto & [column_name, column_identifier] : table_expression_data.getColumnNameToIdentifier()) + { + auto position = query_plan.getCurrentDataStream().header.getPositionByName(column_name); + const auto * node_to_rename = rename_actions_dag->getOutputs()[position]; + rename_actions_dag->getOutputs()[position] = &rename_actions_dag->addAlias(*node_to_rename, column_identifier); + } + + auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), rename_actions_dag); + rename_step->setStepDescription("Change column names to column identifiers"); + query_plan.addStep(std::move(rename_step)); + + return query_plan; +} + +QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContextPtr & planner_context) +{ + auto & join_node = join_tree_node->as(); + + auto left_plan = buildQueryPlanForJoinTreeNode(join_node.getLeftTableExpression(), + select_query_info, + select_query_options, + planner_context); + auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + auto right_plan = buildQueryPlanForJoinTreeNode(join_node.getRightTableExpression(), + select_query_info, + select_query_options, + planner_context); + auto right_plan_output_columns = right_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + if (join_node.getStrictness() == JoinStrictness::Asof) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "JOIN {} ASOF is not supported", + join_node.formatASTForErrorMessage()); + + JoinClausesAndActions join_clauses_and_actions; + + if (join_node.isOnJoinExpression()) + { + join_clauses_and_actions = buildJoinClausesAndActions(left_plan_output_columns, + right_plan_output_columns, + join_tree_node, + planner_context); + + join_clauses_and_actions.left_join_expressions_actions->projectInput(); + auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); + left_join_expressions_actions_step->setStepDescription("JOIN actions"); + left_plan.addStep(std::move(left_join_expressions_actions_step)); + + join_clauses_and_actions.right_join_expressions_actions->projectInput(); + auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); + right_join_expressions_actions_step->setStepDescription("JOIN actions"); + right_plan.addStep(std::move(right_join_expressions_actions_step)); + } + + std::unordered_map left_plan_column_name_to_cast_type; + std::unordered_map right_plan_column_name_to_cast_type; + + if (join_node.isUsingJoinExpression()) + { + auto & join_node_using_columns_list = join_node.getJoinExpression()->as(); + for (auto & join_node_using_node : join_node_using_columns_list.getNodes()) + { + auto & join_node_using_column_node = join_node_using_node->as(); + auto & inner_columns_list = join_node_using_column_node.getExpressionOrThrow()->as(); + + auto & left_inner_column_node = inner_columns_list.getNodes().at(0); + auto & left_inner_column = left_inner_column_node->as(); + + auto & right_inner_column_node = inner_columns_list.getNodes().at(1); + auto & right_inner_column = right_inner_column_node->as(); + + const auto & join_node_using_column_node_type = join_node_using_column_node.getColumnType(); + if (!left_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) + { + auto left_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(left_inner_column_node); + left_plan_column_name_to_cast_type.emplace(left_inner_column_identifier, join_node_using_column_node_type); + } + + if (!right_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) + { + auto right_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(right_inner_column_node); + right_plan_column_name_to_cast_type.emplace(right_inner_column_identifier, join_node_using_column_node_type); + } + } + } + + auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map & plan_column_name_to_cast_type) + { + auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); + + for (auto & output_node : cast_actions_dag->getOutputs()) + { + auto it = plan_column_name_to_cast_type.find(output_node->result_name); + if (it == plan_column_name_to_cast_type.end()) + continue; + + const auto & cast_type = it->second; + auto cast_type_name = cast_type->getName(); + Field cast_type_constant_value(cast_type_name); + + ColumnWithTypeAndName column; + column.name = calculateConstantActionNodeName(cast_type_constant_value); + column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); + column.type = std::make_shared(); + + const auto * cast_type_constant_node = &cast_actions_dag->addColumn(std::move(column)); + + FunctionCastBase::Diagnostic diagnostic = {output_node->result_name, output_node->result_name}; + FunctionOverloadResolverPtr func_builder_cast + = CastInternalOverloadResolver::createImpl(std::move(diagnostic)); + + ActionsDAG::NodeRawConstPtrs children = {output_node, cast_type_constant_node}; + output_node = &cast_actions_dag->addFunction(func_builder_cast, std::move(children), output_node->result_name); + } + + cast_actions_dag->projectInput(); + auto cast_join_columns_step + = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); + cast_join_columns_step->setStepDescription("Cast JOIN USING columns"); + plan_to_add_cast.addStep(std::move(cast_join_columns_step)); + }; + + if (!left_plan_column_name_to_cast_type.empty()) + join_cast_plan_output_nodes(left_plan, left_plan_column_name_to_cast_type); + + if (!right_plan_column_name_to_cast_type.empty()) + join_cast_plan_output_nodes(right_plan, right_plan_column_name_to_cast_type); + + const auto & query_context = planner_context->getQueryContext(); + JoinKind join_kind = join_node.getKind(); + bool join_use_nulls = query_context->getSettingsRef().join_use_nulls; + auto to_nullable_function = FunctionFactory::instance().get("toNullable", query_context); + + auto join_cast_plan_columns_to_nullable = [&](QueryPlan & plan_to_add_cast) + { + auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); + + for (auto & output_node : cast_actions_dag->getOutputs()) + { + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name)) + output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); + } + + cast_actions_dag->projectInput(); + auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); + cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable"); + plan_to_add_cast.addStep(std::move(cast_join_columns_step)); + }; + + if (join_use_nulls) + { + if (isFull(join_kind)) + { + join_cast_plan_columns_to_nullable(left_plan); + join_cast_plan_columns_to_nullable(right_plan); + } + else if (isLeft(join_kind)) + { + join_cast_plan_columns_to_nullable(right_plan); + } + else if (isRight(join_kind)) + { + join_cast_plan_columns_to_nullable(left_plan); + } + } + + auto table_join = std::make_shared(); + table_join->getTableJoin() = join_node.toASTTableJoin()->as(); + if (join_node.getKind() == JoinKind::Comma) + table_join->getTableJoin().kind = JoinKind::Cross; + table_join->getTableJoin().strictness = JoinStrictness::All; + + if (join_node.isOnJoinExpression()) + { + const auto & join_clauses = join_clauses_and_actions.join_clauses; + auto & table_join_clauses = table_join->getClauses(); + + for (const auto & join_clause : join_clauses) + { + table_join_clauses.emplace_back(); + auto & table_join_clause = table_join_clauses.back(); + + const auto & join_clause_left_key_nodes = join_clause.getLeftKeyNodes(); + const auto & join_clause_right_key_nodes = join_clause.getRightKeyNodes(); + + size_t join_clause_key_nodes_size = join_clause_left_key_nodes.size(); + assert(join_clause_key_nodes_size == join_clause_right_key_nodes.size()); + + for (size_t i = 0; i < join_clause_key_nodes_size; ++i) + { + table_join_clause.key_names_left.push_back(join_clause_left_key_nodes[i]->result_name); + table_join_clause.key_names_right.push_back(join_clause_right_key_nodes[i]->result_name); + } + + const auto & join_clause_get_left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); + if (!join_clause_get_left_filter_condition_nodes.empty()) + { + if (join_clause_get_left_filter_condition_nodes.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} left filter conditions size must be 1. Actual {}", + join_node.formatASTForErrorMessage(), + join_clause_get_left_filter_condition_nodes.size()); + + const auto & join_clause_left_filter_condition_name = join_clause_get_left_filter_condition_nodes[0]->result_name; + table_join_clause.analyzer_left_filter_condition_column_name = join_clause_left_filter_condition_name; + } + + const auto & join_clause_get_right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); + if (!join_clause_get_right_filter_condition_nodes.empty()) + { + if (join_clause_get_right_filter_condition_nodes.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} right filter conditions size must be 1. Actual {}", + join_node.formatASTForErrorMessage(), + join_clause_get_right_filter_condition_nodes.size()); + + const auto & join_clause_right_filter_condition_name = join_clause_get_right_filter_condition_nodes[0]->result_name; + table_join_clause.analyzer_right_filter_condition_column_name = join_clause_right_filter_condition_name; + } + } + } + else if (join_node.isUsingJoinExpression()) + { + auto & table_join_clauses = table_join->getClauses(); + table_join_clauses.emplace_back(); + auto & table_join_clause = table_join_clauses.back(); + + auto & using_list = join_node.getJoinExpression()->as(); + + for (auto & join_using_node : using_list.getNodes()) + { + auto & join_using_column_node = join_using_node->as(); + if (!join_using_column_node.getExpression() || + join_using_column_node.getExpression()->getNodeType() != QueryTreeNodeType::LIST) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} column in USING does not have inner columns", + join_node.formatASTForErrorMessage()); + + auto & using_join_columns_list = join_using_column_node.getExpression()->as(); + auto & using_join_left_join_column_node = using_join_columns_list.getNodes().at(0); + auto & using_join_right_join_column_node = using_join_columns_list.getNodes().at(1); + + auto left_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_left_join_column_node); + auto right_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_right_join_column_node); + + table_join_clause.key_names_left.push_back(left_column_identifier); + table_join_clause.key_names_right.push_back(right_column_identifier); + } + } + + auto left_table_names = left_plan.getCurrentDataStream().header.getNames(); + NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); + + auto columns_from_joined_table = right_plan.getCurrentDataStream().header.getNamesAndTypesList(); + table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); + + for (auto & column_from_joined_table : columns_from_joined_table) + { + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name)) + table_join->addJoinedColumn(column_from_joined_table); + } + + size_t max_block_size = query_context->getSettingsRef().max_block_size; + size_t max_streams = query_context->getSettingsRef().max_threads; + + JoinPtr join_ptr = std::make_shared(table_join, right_plan.getCurrentDataStream().header, false /*any_take_last_row*/); + QueryPlanStepPtr join_step = std::make_unique( + left_plan.getCurrentDataStream(), + right_plan.getCurrentDataStream(), + join_ptr, + max_block_size, + max_streams, + false /*optimize_read_in_order*/); + + join_step->setStepDescription(fmt::format("JOIN {}", JoinPipelineType::FillRightFirst)); + + std::vector plans; + plans.emplace_back(std::make_unique(std::move(left_plan))); + plans.emplace_back(std::make_unique(std::move(right_plan))); + + auto result_plan = QueryPlan(); + result_plan.unitePlans(std::move(join_step), {std::move(plans)}); + + auto drop_unused_columns_after_join_actions_dag = std::make_shared(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs updated_outputs; + std::unordered_set updated_outputs_names; + + for (auto & output : drop_unused_columns_after_join_actions_dag->getOutputs()) + { + if (updated_outputs_names.contains(output->result_name) || !planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output->result_name)) + continue; + + updated_outputs.push_back(output); + updated_outputs_names.insert(output->result_name); + } + + drop_unused_columns_after_join_actions_dag->getOutputs() = std::move(updated_outputs); + + auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentDataStream(), std::move(drop_unused_columns_after_join_actions_dag)); + drop_unused_columns_after_join_transform_step->setStepDescription("DROP unused columns after JOIN"); + result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); + + return result_plan; +} + +QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContextPtr & planner_context) +{ + auto & array_join_node = table_expression->as(); + + auto left_plan = buildQueryPlanForJoinTreeNode(array_join_node.getTableExpression(), + select_query_info, + select_query_options, + planner_context); + auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + + ActionsDAGPtr array_join_action_dag = std::make_shared(left_plan_output_columns); + PlannerActionsVisitor actions_visitor(planner_context); + + NameSet array_join_columns; + for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) + { + auto & array_join_expression_column = array_join_expression->as(); + const auto & array_join_column_name = array_join_expression_column.getColumnName(); + array_join_columns.insert(array_join_column_name); + + auto expression_dag_index_nodes = actions_visitor.visit(array_join_action_dag, array_join_expression_column.getExpressionOrThrow()); + for (auto & expression_dag_index_node : expression_dag_index_nodes) + { + const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_name); + array_join_action_dag->getOutputs().push_back(array_join_column_node); + } + } + + array_join_action_dag->projectInput(); + auto array_join_actions = std::make_unique(left_plan.getCurrentDataStream(), array_join_action_dag); + array_join_actions->setStepDescription("ARRAY JOIN actions"); + left_plan.addStep(std::move(array_join_actions)); + + auto array_join_action = std::make_shared(array_join_columns, array_join_node.isLeft(), planner_context->getQueryContext()); + auto array_join_step = std::make_unique(left_plan.getCurrentDataStream(), std::move(array_join_action)); + array_join_step->setStepDescription("ARRAY JOIN"); + left_plan.addStep(std::move(array_join_step)); + + return left_plan; +} + +} + +QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContextPtr & planner_context) +{ + auto join_tree_node_type = join_tree_node->getNodeType(); + + switch (join_tree_node_type) + { + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::UNION: + [[fallthrough]]; + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + return buildQueryPlanForTableExpression(join_tree_node, select_query_info, select_query_options, planner_context); + } + case QueryTreeNodeType::JOIN: + { + return buildQueryPlanForJoinNode(join_tree_node, select_query_info, select_query_options, planner_context); + } + case QueryTreeNodeType::ARRAY_JOIN: + { + return buildQueryPlanForArrayJoinNode(join_tree_node, select_query_info, select_query_options, planner_context); + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected query, table, table function, join or array join query node. Actual {}", + join_tree_node->formatASTForErrorMessage()); + } + } +} + +} diff --git a/src/Planner/PlannerJoinTree.h b/src/Planner/PlannerJoinTree.h new file mode 100644 index 00000000000..c93b71e0df1 --- /dev/null +++ b/src/Planner/PlannerJoinTree.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +#include + +#include + +#include + +namespace DB +{ + +/// Build query plan for query JOIN TREE node +QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContextPtr & planner_context); + +} From 3782aed7a9da30f2392ad2f49002bf8a71c8a293 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 27 Sep 2022 12:53:25 +0200 Subject: [PATCH 106/188] Separate planner expression analysis --- src/Planner/Planner.cpp | 469 +++----------------- src/Planner/PlannerExpressionAnalysis.cpp | 503 ++++++++++++++++++++++ src/Planner/PlannerExpressionAnalysis.h | 175 ++++++++ 3 files changed, 736 insertions(+), 411 deletions(-) create mode 100644 src/Planner/PlannerExpressionAnalysis.cpp create mode 100644 src/Planner/PlannerExpressionAnalysis.h diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 61d97daf8bc..74c5112cc9a 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -60,6 +60,7 @@ #include #include #include +#include namespace DB { @@ -290,376 +291,28 @@ void Planner::buildQueryPlanIfNeeded() collectSets(query_tree, *planner_context); query_plan = buildQueryPlanForJoinTreeNode(query_node.getJoinTree(), select_query_info, select_query_options, planner_context); + auto expression_analysis_result = buildExpressionAnalysisResult(query_tree, query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), planner_context); - ActionsChain actions_chain; - std::optional where_action_step_index; - std::string where_filter_action_node_name; - - if (query_node.hasWhere()) + if (expression_analysis_result.hasWhere()) { - const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & where_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - - auto where_actions = buildActionsDAGFromExpressionNode(query_node.getWhere(), where_input, planner_context); - where_filter_action_node_name = where_actions->getOutputs().at(0)->result_name; - actions_chain.addStep(std::make_unique(std::move(where_actions))); - where_action_step_index = actions_chain.getLastStepIndex(); - } - - auto aggregate_function_nodes = collectAggregateFunctionNodes(query_tree); - AggregateDescriptions aggregates_descriptions = extractAggregateDescriptions(aggregate_function_nodes, *planner_context); - ColumnsWithTypeAndName aggregates_columns; - aggregates_columns.reserve(aggregates_descriptions.size()); - for (auto & aggregate_description : aggregates_descriptions) - aggregates_columns.emplace_back(nullptr, aggregate_description.function->getReturnType(), aggregate_description.column_name); - - Names aggregation_keys; - std::optional aggregate_step_index; - - const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & group_by_input = chain_available_output_columns ? *chain_available_output_columns - : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - - /// Only aggregation keys, and aggregates are available for next steps after GROUP BY step - ActionsDAGPtr group_by_actions_dag = std::make_shared(group_by_input); - group_by_actions_dag->getOutputs().clear(); - std::unordered_set group_by_actions_dag_output_nodes_names; - - PlannerActionsVisitor actions_visitor(planner_context); - GroupingSetsParamsList grouping_sets_parameters_list; - bool group_by_with_constant_keys = false; - bool disable_grouping_sets = false; - - if (query_node.hasGroupBy()) - { - if (query_node.isGroupByWithGroupingSets()) - { - for (auto & grouping_set_keys_list_node : query_node.getGroupBy().getNodes()) - { - auto & grouping_set_keys_list_node_typed = grouping_set_keys_list_node->as(); - grouping_sets_parameters_list.emplace_back(); - auto & grouping_sets_parameters = grouping_sets_parameters_list.back(); - - for (auto & grouping_set_key_node : grouping_set_keys_list_node_typed.getNodes()) - { - group_by_with_constant_keys |= grouping_set_key_node->hasConstantValue(); - - auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, grouping_set_key_node); - aggregation_keys.reserve(expression_dag_nodes.size()); - - for (auto & expression_dag_node : expression_dag_nodes) - { - grouping_sets_parameters.used_keys.push_back(expression_dag_node->result_name); - if (group_by_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) - continue; - - aggregation_keys.push_back(expression_dag_node->result_name); - group_by_actions_dag->getOutputs().push_back(expression_dag_node); - group_by_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); - } - } - } - - for (auto & grouping_sets_parameter : grouping_sets_parameters_list) - { - NameSet grouping_sets_used_keys; - Names grouping_sets_keys; - - for (auto & key : grouping_sets_parameter.used_keys) - { - auto [_, inserted] = grouping_sets_used_keys.insert(key); - if (inserted) - grouping_sets_keys.push_back(key); - } - - for (auto & key : aggregation_keys) - { - if (grouping_sets_used_keys.contains(key)) - continue; - - grouping_sets_parameter.missing_keys.push_back(key); - } - - grouping_sets_parameter.used_keys = std::move(grouping_sets_keys); - } - - /// It is expected by execution layer that if there are only 1 grouping sets it will be removed - if (grouping_sets_parameters_list.size() == 1) - { - disable_grouping_sets = true; - grouping_sets_parameters_list.clear(); - } - } - else - { - for (auto & group_by_key_node : query_node.getGroupBy().getNodes()) - group_by_with_constant_keys |= group_by_key_node->hasConstantValue(); - - auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, query_node.getGroupByNode()); - aggregation_keys.reserve(expression_dag_nodes.size()); - - for (auto & expression_dag_node : expression_dag_nodes) - { - if (group_by_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) - continue; - - aggregation_keys.push_back(expression_dag_node->result_name); - group_by_actions_dag->getOutputs().push_back(expression_dag_node); - group_by_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); - } - } - } - - if (!aggregate_function_nodes.empty()) - { - for (auto & aggregate_function_node : aggregate_function_nodes) - { - auto & aggregate_function_node_typed = aggregate_function_node->as(); - for (const auto & aggregate_function_node_argument : aggregate_function_node_typed.getArguments().getNodes()) - { - auto expression_dag_nodes = actions_visitor.visit(group_by_actions_dag, aggregate_function_node_argument); - for (auto & expression_dag_node : expression_dag_nodes) - { - if (group_by_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) - continue; - - group_by_actions_dag->getOutputs().push_back(expression_dag_node); - group_by_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); - } - } - } - } - - if (!group_by_actions_dag->getOutputs().empty()) - { - /** For non ordinary GROUP BY we add virtual __grouping_set column - * With set number, which is used as an additional key at the stage of merging aggregating data. - */ - if (query_node.isGroupByWithRollup() || query_node.isGroupByWithCube() || (query_node.isGroupByWithGroupingSets() && !disable_grouping_sets)) - aggregates_columns.emplace_back(nullptr, std::make_shared(), "__grouping_set"); - - resolveGroupingFunctions(query_tree, aggregation_keys, grouping_sets_parameters_list, *planner_context); - auto aggregate_step = std::make_unique(std::move(group_by_actions_dag), ActionsChainStep::AvailableOutputColumnsStrategy::OUTPUT_NODES, aggregates_columns); - actions_chain.addStep(std::move(aggregate_step)); - aggregate_step_index = actions_chain.getLastStepIndex(); - } - - std::optional having_action_step_index; - std::string having_filter_action_node_name; - - if (query_node.hasHaving()) - { - chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & having_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - - auto having_actions = buildActionsDAGFromExpressionNode(query_node.getHaving(), having_input, planner_context); - having_filter_action_node_name = having_actions->getOutputs().at(0)->result_name; - actions_chain.addStep(std::make_unique(std::move(having_actions))); - having_action_step_index = actions_chain.getLastStepIndex(); - } - - auto window_function_nodes = collectWindowFunctionNodes(query_tree); - auto window_descriptions = extractWindowDescriptions(window_function_nodes, *planner_context); - std::optional before_window_step_index; - - if (!window_function_nodes.empty()) - { - chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & window_input = chain_available_output_columns ? *chain_available_output_columns - : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - - ActionsDAGPtr before_window_actions_dag = std::make_shared(window_input); - before_window_actions_dag->getOutputs().clear(); - - std::unordered_set before_window_actions_dag_output_nodes_names; - - for (auto & window_function_node : window_function_nodes) - { - auto & window_function_node_typed = window_function_node->as(); - auto & window_node = window_function_node_typed.getWindowNode()->as(); - - auto expression_dag_nodes = actions_visitor.visit(before_window_actions_dag, window_function_node_typed.getArgumentsNode()); - aggregation_keys.reserve(expression_dag_nodes.size()); - - for (auto & expression_dag_node : expression_dag_nodes) - { - if (before_window_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) - continue; - - before_window_actions_dag->getOutputs().push_back(expression_dag_node); - before_window_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); - } - - expression_dag_nodes = actions_visitor.visit(before_window_actions_dag, window_node.getPartitionByNode()); - aggregation_keys.reserve(expression_dag_nodes.size()); - - for (auto & expression_dag_node : expression_dag_nodes) - { - if (before_window_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) - continue; - - before_window_actions_dag->getOutputs().push_back(expression_dag_node); - before_window_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); - } - - /** We add only sort column sort expression in before WINDOW actions DAG. - * WITH fill expressions must be constant nodes. - */ - auto & order_by_node_list = window_node.getOrderBy(); - for (auto & sort_node : order_by_node_list.getNodes()) - { - auto & sort_node_typed = sort_node->as(); - expression_dag_nodes = actions_visitor.visit(before_window_actions_dag, sort_node_typed.getExpression()); - - for (auto & expression_dag_node : expression_dag_nodes) - { - if (before_window_actions_dag_output_nodes_names.contains(expression_dag_node->result_name)) - continue; - - before_window_actions_dag->getOutputs().push_back(expression_dag_node); - before_window_actions_dag_output_nodes_names.insert(expression_dag_node->result_name); - } - } - } - - ColumnsWithTypeAndName window_functions_additional_columns; - - for (auto & window_description : window_descriptions) - for (auto & window_function : window_description.window_functions) - window_functions_additional_columns.emplace_back(nullptr, window_function.aggregate_function->getReturnType(), window_function.column_name); - - auto before_window_step = std::make_unique(std::move(before_window_actions_dag), - ActionsChainStep::AvailableOutputColumnsStrategy::ALL_NODES, - window_functions_additional_columns); - actions_chain.addStep(std::move(before_window_step)); - before_window_step_index = actions_chain.getLastStepIndex(); - } - - chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), projection_input, planner_context); - - auto projection_columns = query_node.getProjectionColumns(); - size_t projection_columns_size = projection_columns.size(); - - Names projection_action_names; - NamesWithAliases projection_action_names_with_display_aliases; - projection_action_names_with_display_aliases.reserve(projection_columns_size); - - auto & projection_actions_outputs = projection_actions->getOutputs(); - size_t projection_outputs_size = projection_actions_outputs.size(); - - if (projection_columns_size != projection_outputs_size) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "QueryTree projection nodes size mismatch. Expected {}. Actual {}", - projection_outputs_size, - projection_columns_size); - - for (size_t i = 0; i < projection_outputs_size; ++i) - { - auto & projection_column = projection_columns[i]; - const auto * projection_node = projection_actions_outputs[i]; - const auto & projection_node_name = projection_node->result_name; - - projection_action_names.push_back(projection_node_name); - projection_action_names_with_display_aliases.push_back({projection_node_name, projection_column.name}); - } - - auto projection_actions_step = std::make_unique(std::move(projection_actions)); - actions_chain.addStep(std::move(projection_actions_step)); - size_t projection_step_index = actions_chain.getLastStepIndex(); - - std::optional before_order_by_step_index; - if (query_node.hasOrderBy()) - { - chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & order_by_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - - ActionsDAGPtr before_order_by_actions_dag = std::make_shared(order_by_input); - auto & before_order_by_actions_dag_outputs = before_order_by_actions_dag->getOutputs(); - before_order_by_actions_dag_outputs.clear(); - - std::unordered_set before_order_by_actions_dag_outputs_node_names; - - /** We add only sort node sort expression in before ORDER BY actions DAG. - * WITH fill expressions must be constant nodes. - */ - auto & order_by_node_list = query_node.getOrderBy(); - for (auto & sort_node : order_by_node_list.getNodes()) - { - auto & sort_node_typed = sort_node->as(); - auto expression_dag_nodes = actions_visitor.visit(before_order_by_actions_dag, sort_node_typed.getExpression()); - - for (auto & action_dag_node : expression_dag_nodes) - { - if (before_order_by_actions_dag_outputs_node_names.contains(action_dag_node->result_name)) - continue; - - before_order_by_actions_dag_outputs.push_back(action_dag_node); - before_order_by_actions_dag_outputs_node_names.insert(action_dag_node->result_name); - } - } - - auto actions_step_before_order_by = std::make_unique(std::move(before_order_by_actions_dag)); - actions_chain.addStep(std::move(actions_step_before_order_by)); - before_order_by_step_index = actions_chain.getLastStepIndex(); - } - - std::optional before_limit_by_step_index; - Names limit_by_columns_names; - - if (query_node.hasLimitBy()) - { - chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & limit_by_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - auto limit_by_actions = buildActionsDAGFromExpressionNode(query_node.getLimitByNode(), limit_by_input, planner_context); - - limit_by_columns_names.reserve(limit_by_actions->getOutputs().size()); - for (auto & output_node : limit_by_actions->getOutputs()) - limit_by_columns_names.push_back(output_node->result_name); - - auto actions_step_before_limit_by = std::make_unique(std::move(limit_by_actions)); - actions_chain.addStep(std::move(actions_step_before_limit_by)); - before_limit_by_step_index = actions_chain.getLastStepIndex(); - } - - chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & project_names_input = chain_available_output_columns ? *chain_available_output_columns : query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - auto project_names_actions = std::make_shared(project_names_input); - - project_names_actions->project(projection_action_names_with_display_aliases); - actions_chain.addStep(std::make_unique(std::move(project_names_actions))); - size_t project_names_action_step_index = actions_chain.getLastStepIndex(); - - // std::cout << "Chain dump before finalize" << std::endl; - // std::cout << actions_chain.dump() << std::endl; - - actions_chain.finalize(); - - // std::cout << "Chain dump after finalize" << std::endl; - // std::cout << actions_chain.dump() << std::endl; - - if (where_action_step_index) - { - auto & where_actions_chain_node = actions_chain.at(*where_action_step_index); - bool remove_filter = !where_actions_chain_node->getChildRequiredOutputColumnsNames().contains(where_filter_action_node_name); + const auto & where_analysis_result = expression_analysis_result.getWhere(); auto where_step = std::make_unique(query_plan.getCurrentDataStream(), - where_actions_chain_node->getActions(), - where_filter_action_node_name, - remove_filter); + where_analysis_result.filter_actions, + where_analysis_result.filter_column_name, + where_analysis_result.remove_filter_column); where_step->setStepDescription("WHERE"); query_plan.addStep(std::move(where_step)); } bool having_executed = false; - if (!aggregates_descriptions.empty() || query_node.hasGroupBy()) + if (expression_analysis_result.hasAggregation()) { - if (aggregate_step_index) + const auto & aggregation_analysis_result = expression_analysis_result.getAggregation(); + + if (aggregation_analysis_result.before_aggregation_actions) { - auto & aggregate_actions_chain_node = actions_chain.at(*aggregate_step_index); - auto expression_before_aggregation = std::make_unique(query_plan.getCurrentDataStream(), - aggregate_actions_chain_node->getActions()); + auto expression_before_aggregation = std::make_unique(query_plan.getCurrentDataStream(), aggregation_analysis_result.before_aggregation_actions); expression_before_aggregation->setStepDescription("Before GROUP BY"); query_plan.addStep(std::move(expression_before_aggregation)); } @@ -679,8 +332,8 @@ void Planner::buildQueryPlanIfNeeded() settings.totals_mode != TotalsMode::AFTER_HAVING_EXCLUSIVE; Aggregator::Params aggregator_params = Aggregator::Params( - aggregation_keys, - aggregates_descriptions, + aggregation_analysis_result.aggregation_keys, + aggregation_analysis_result.aggregate_descriptions, aggregate_overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode, @@ -688,8 +341,8 @@ void Planner::buildQueryPlanIfNeeded() settings.group_by_two_level_threshold_bytes, settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set - || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && aggregation_keys.empty() - && group_by_with_constant_keys), + || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && aggregation_analysis_result.aggregation_keys.empty() + && aggregation_analysis_result.group_by_with_constant_keys), planner_context->getQueryContext()->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data, @@ -731,7 +384,7 @@ void Planner::buildQueryPlanIfNeeded() auto aggregating_step = std::make_unique( query_plan.getCurrentDataStream(), aggregator_params, - std::move(grouping_sets_parameters_list), + aggregation_analysis_result.grouping_sets_parameters_list, aggregate_final, settings.max_block_size, settings.aggregation_in_order_max_block_bytes, @@ -757,25 +410,16 @@ void Planner::buildQueryPlanIfNeeded() if (query_node.isGroupByWithTotals()) { - bool remove_having_filter = false; - std::shared_ptr having_actions; - - if (having_action_step_index) - { - auto & having_actions_chain_node = actions_chain.at(*having_action_step_index); - remove_having_filter = !having_actions_chain_node->getChildRequiredOutputColumnsNames().contains(having_filter_action_node_name); - having_actions = having_actions_chain_node->getActions(); - having_executed = true; - } - + const auto & having_analysis_result = expression_analysis_result.getHaving(); bool final = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube(); + auto totals_having_step = std::make_unique( query_plan.getCurrentDataStream(), - aggregates_descriptions, + aggregation_analysis_result.aggregate_descriptions, aggregate_overflow_row, - having_actions, - having_filter_action_node_name, - remove_having_filter, + having_analysis_result.filter_actions, + having_analysis_result.filter_column_name, + having_analysis_result.remove_filter_column, settings.totals_mode, settings.totals_auto_threshold, final); @@ -784,27 +428,32 @@ void Planner::buildQueryPlanIfNeeded() } } - if (!having_executed && having_action_step_index) + if (!having_executed && expression_analysis_result.hasHaving()) { - auto & having_actions_chain_node = actions_chain.at(*having_action_step_index); - bool remove_filter = !having_actions_chain_node->getChildRequiredOutputColumnsNames().contains(having_filter_action_node_name); + const auto & having_analysis_result = expression_analysis_result.getHaving(); + auto having_step = std::make_unique(query_plan.getCurrentDataStream(), - having_actions_chain_node->getActions(), - having_filter_action_node_name, - remove_filter); + having_analysis_result.filter_actions, + having_analysis_result.filter_column_name, + having_analysis_result.remove_filter_column); having_step->setStepDescription("HAVING"); query_plan.addStep(std::move(having_step)); } - if (before_window_step_index) + if (expression_analysis_result.hasWindow()) { - auto & before_window_actions_chain_node = actions_chain.at(*before_window_step_index); - auto expression_step_before_window = std::make_unique(query_plan.getCurrentDataStream(), - before_window_actions_chain_node->getActions()); - expression_step_before_window->setStepDescription("Before WINDOW"); - query_plan.addStep(std::move(expression_step_before_window)); + const auto & window_analysis_result = expression_analysis_result.getWindow(); + if (window_analysis_result.before_window_actions) + { + auto expression_step_before_window = std::make_unique(query_plan.getCurrentDataStream(), window_analysis_result.before_window_actions); + expression_step_before_window->setStepDescription("Before WINDOW"); + query_plan.addStep(std::move(expression_step_before_window)); + } + + auto window_descriptions = window_analysis_result.window_descriptions; sortWindowDescriptions(window_descriptions); + size_t window_descriptions_size = window_descriptions.size(); const auto & settings = query_context->getSettingsRef(); @@ -843,9 +492,8 @@ void Planner::buildQueryPlanIfNeeded() } } - auto & projection_actions_chain_node = actions_chain.at(projection_step_index); - auto expression_step_projection = std::make_unique(query_plan.getCurrentDataStream(), - projection_actions_chain_node->getActions()); + const auto & projection_analysis_result = expression_analysis_result.getProjection(); + auto expression_step_projection = std::make_unique(query_plan.getCurrentDataStream(), projection_analysis_result.projection_actions); expression_step_projection->setStepDescription("Projection"); query_plan.addStep(std::move(expression_step_projection)); @@ -861,7 +509,7 @@ void Planner::buildQueryPlanIfNeeded() query_plan.getCurrentDataStream(), limits, limit_hint_for_distinct, - projection_action_names, + projection_analysis_result.projection_column_names, pre_distinct, settings.optimize_distinct_in_order); @@ -873,11 +521,10 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(distinct_step)); } - if (before_order_by_step_index) + if (expression_analysis_result.hasSort()) { - auto & before_order_by_actions_chain_node = actions_chain.at(*before_order_by_step_index); - auto expression_step_before_order_by = std::make_unique(query_plan.getCurrentDataStream(), - before_order_by_actions_chain_node->getActions()); + const auto & sort_analysis_result = expression_analysis_result.getSort(); + auto expression_step_before_order_by = std::make_unique(query_plan.getCurrentDataStream(), sort_analysis_result.before_order_by_actions); expression_step_before_order_by->setStepDescription("Before ORDER BY"); query_plan.addStep(std::move(expression_step_before_order_by)); } @@ -888,7 +535,6 @@ void Planner::buildQueryPlanIfNeeded() if (query_node.hasOrderBy()) { sort_description = extractSortDescription(query_node.getOrderByNode(), *planner_context); - String sort_description_dump = dumpSortDescription(sort_description); UInt64 limit = 0; @@ -950,8 +596,10 @@ void Planner::buildQueryPlanIfNeeded() for (auto & interpolate_node : interpolate_list_nodes) { auto & interpolate_node_typed = interpolate_node->as(); - auto expression_to_interpolate_expression_nodes = actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getExpression()); - auto interpolate_expression_nodes = actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); + + PlannerActionsVisitor planner_actions_visitor(planner_context); + auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getExpression()); + auto interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); if (expression_to_interpolate_expression_nodes.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression to interpolate expected to have single action node"); @@ -998,17 +646,13 @@ void Planner::buildQueryPlanIfNeeded() } } - if (before_limit_by_step_index) + if (expression_analysis_result.hasLimitBy()) { - auto & before_limit_by_actions_chain_node = actions_chain.at(*before_limit_by_step_index); - auto expression_step_before_limit_by = std::make_unique(query_plan.getCurrentDataStream(), - before_limit_by_actions_chain_node->getActions()); + const auto & limit_by_analysis_result = expression_analysis_result.getLimitBy(); + auto expression_step_before_limit_by = std::make_unique(query_plan.getCurrentDataStream(), limit_by_analysis_result.before_limit_by_actions); expression_step_before_limit_by->setStepDescription("Before LIMIT BY"); query_plan.addStep(std::move(expression_step_before_limit_by)); - } - if (query_node.hasLimitByLimit() && query_node.hasLimitBy()) - { /// Constness of LIMIT BY limit is validated during query analysis stage UInt64 limit_by_limit = query_node.getLimitByLimit()->getConstantValue().getValue().safeGet(); UInt64 limit_by_offset = 0; @@ -1019,7 +663,10 @@ void Planner::buildQueryPlanIfNeeded() limit_by_offset = query_node.getLimitByOffset()->getConstantValue().getValue().safeGet(); } - auto limit_by_step = std::make_unique(query_plan.getCurrentDataStream(), limit_by_limit, limit_by_offset, limit_by_columns_names); + auto limit_by_step = std::make_unique(query_plan.getCurrentDataStream(), + limit_by_limit, + limit_by_offset, + limit_by_analysis_result.limit_by_column_names); query_plan.addStep(std::move(limit_by_step)); } @@ -1077,7 +724,7 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(offsets_step)); } - auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), actions_chain[project_names_action_step_index]->getActions()); + auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), projection_analysis_result.project_names_actions); projection_step->setStepDescription("Project names"); query_plan.addStep(std::move(projection_step)); diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp new file mode 100644 index 00000000000..f428668baef --- /dev/null +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -0,0 +1,503 @@ +#include + +#include + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +/** Construct filter analysis result for filter expression node + * Actions before filter are added into into actions chain. + * It is client responsibility to update filter analysis result if filter column must be removed after chain is finalized. + */ +FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_node, + const ColumnsWithTypeAndName & join_tree_input_columns, + const PlannerContextPtr & planner_context, + ActionsChain & actions_chain) +{ + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & filter_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + + FilterAnalysisResult result; + + result.filter_actions = buildActionsDAGFromExpressionNode(filter_expression_node, filter_input, planner_context); + result.filter_column_name = result.filter_actions->getOutputs().at(0)->result_name; + actions_chain.addStep(std::make_unique(result.filter_actions)); + + return result; +} + +/** Construct aggregation analysis result if query tree has GROUP BY or aggregates. + * Actions before aggregation are added into actions chain, if result is not null optional. + */ +std::optional analyzeAggregation(QueryTreeNodePtr & query_tree, + const ColumnsWithTypeAndName & join_tree_input_columns, + const PlannerContextPtr & planner_context, + ActionsChain & actions_chain) +{ + auto & query_node = query_tree->as(); + + auto aggregate_function_nodes = collectAggregateFunctionNodes(query_tree); + auto aggregates_descriptions = extractAggregateDescriptions(aggregate_function_nodes, *planner_context); + + ColumnsWithTypeAndName aggregates_columns; + aggregates_columns.reserve(aggregates_descriptions.size()); + for (auto & aggregate_description : aggregates_descriptions) + aggregates_columns.emplace_back(nullptr, aggregate_description.function->getReturnType(), aggregate_description.column_name); + + Names aggregation_keys; + + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & group_by_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + + ActionsDAGPtr before_aggregation_actions = std::make_shared(group_by_input); + before_aggregation_actions->getOutputs().clear(); + + std::unordered_set before_aggregation_actions_output_node_names; + + GroupingSetsParamsList grouping_sets_parameters_list; + bool group_by_with_constant_keys = false; + bool disable_grouping_sets = false; + + PlannerActionsVisitor actions_visitor(planner_context); + + /// Add expressions from GROUP BY + + if (query_node.hasGroupBy()) + { + if (query_node.isGroupByWithGroupingSets()) + { + for (auto & grouping_set_keys_list_node : query_node.getGroupBy().getNodes()) + { + auto & grouping_set_keys_list_node_typed = grouping_set_keys_list_node->as(); + grouping_sets_parameters_list.emplace_back(); + auto & grouping_sets_parameters = grouping_sets_parameters_list.back(); + + for (auto & grouping_set_key_node : grouping_set_keys_list_node_typed.getNodes()) + { + group_by_with_constant_keys |= grouping_set_key_node->hasConstantValue(); + + auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, grouping_set_key_node); + aggregation_keys.reserve(expression_dag_nodes.size()); + + for (auto & expression_dag_node : expression_dag_nodes) + { + grouping_sets_parameters.used_keys.push_back(expression_dag_node->result_name); + if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name)) + continue; + + aggregation_keys.push_back(expression_dag_node->result_name); + before_aggregation_actions->getOutputs().push_back(expression_dag_node); + before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); + } + } + } + + for (auto & grouping_sets_parameter : grouping_sets_parameters_list) + { + NameSet grouping_sets_used_keys; + Names grouping_sets_keys; + + for (auto & key : grouping_sets_parameter.used_keys) + { + auto [_, inserted] = grouping_sets_used_keys.insert(key); + if (inserted) + grouping_sets_keys.push_back(key); + } + + for (auto & key : aggregation_keys) + { + if (grouping_sets_used_keys.contains(key)) + continue; + + grouping_sets_parameter.missing_keys.push_back(key); + } + + grouping_sets_parameter.used_keys = std::move(grouping_sets_keys); + } + + /// It is expected by execution layer that if there are only 1 grouping sets it will be removed + if (grouping_sets_parameters_list.size() == 1) + { + disable_grouping_sets = true; + grouping_sets_parameters_list.clear(); + } + } + else + { + for (auto & group_by_key_node : query_node.getGroupBy().getNodes()) + group_by_with_constant_keys |= group_by_key_node->hasConstantValue(); + + auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, query_node.getGroupByNode()); + aggregation_keys.reserve(expression_dag_nodes.size()); + + for (auto & expression_dag_node : expression_dag_nodes) + { + if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name)) + continue; + + aggregation_keys.push_back(expression_dag_node->result_name); + before_aggregation_actions->getOutputs().push_back(expression_dag_node); + before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); + } + } + } + + /// Add expressions from aggregate functions arguments + + for (auto & aggregate_function_node : aggregate_function_nodes) + { + auto & aggregate_function_node_typed = aggregate_function_node->as(); + for (const auto & aggregate_function_node_argument : aggregate_function_node_typed.getArguments().getNodes()) + { + auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, aggregate_function_node_argument); + for (auto & expression_dag_node : expression_dag_nodes) + { + if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name)) + continue; + + before_aggregation_actions->getOutputs().push_back(expression_dag_node); + before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); + } + } + } + + if (aggregation_keys.empty() && aggregates_descriptions.empty()) + return {}; + + /** For non ordinary GROUP BY we add virtual __grouping_set column + * With set number, which is used as an additional key at the stage of merging aggregating data. + */ + if (query_node.isGroupByWithRollup() || query_node.isGroupByWithCube() || (query_node.isGroupByWithGroupingSets() && !disable_grouping_sets)) + aggregates_columns.emplace_back(nullptr, std::make_shared(), "__grouping_set"); + + resolveGroupingFunctions(query_tree, aggregation_keys, grouping_sets_parameters_list, *planner_context); + + /// Only aggregation keys and aggregates are available for next steps after GROUP BY step + auto aggregate_step = std::make_unique(before_aggregation_actions, ActionsChainStep::AvailableOutputColumnsStrategy::OUTPUT_NODES, aggregates_columns); + actions_chain.addStep(std::move(aggregate_step)); + + AggregationAnalysisResult aggregation_analysis_result; + aggregation_analysis_result.before_aggregation_actions = before_aggregation_actions; + aggregation_analysis_result.aggregation_keys = std::move(aggregation_keys); + aggregation_analysis_result.aggregate_descriptions = std::move(aggregates_descriptions); + aggregation_analysis_result.grouping_sets_parameters_list = std::move(grouping_sets_parameters_list); + aggregation_analysis_result.group_by_with_constant_keys = group_by_with_constant_keys; + + return aggregation_analysis_result; +} + +/** Construct aggregation analysis result if query tree has window functions. + * Actions before window functions are added into actions chain, if result is not null optional. + */ +std::optional analyzeWindow(QueryTreeNodePtr & query_tree, + const ColumnsWithTypeAndName & join_tree_input_columns, + const PlannerContextPtr & planner_context, + ActionsChain & actions_chain) +{ + auto window_function_nodes = collectWindowFunctionNodes(query_tree); + if (window_function_nodes.empty()) + return {}; + + auto window_descriptions = extractWindowDescriptions(window_function_nodes, *planner_context); + + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & window_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + + PlannerActionsVisitor actions_visitor(planner_context); + + ActionsDAGPtr before_window_actions = std::make_shared(window_input); + before_window_actions->getOutputs().clear(); + + std::unordered_set before_window_actions_output_node_names; + + for (auto & window_function_node : window_function_nodes) + { + auto & window_function_node_typed = window_function_node->as(); + auto & window_node = window_function_node_typed.getWindowNode()->as(); + + auto expression_dag_nodes = actions_visitor.visit(before_window_actions, window_function_node_typed.getArgumentsNode()); + + for (auto & expression_dag_node : expression_dag_nodes) + { + if (before_window_actions_output_node_names.contains(expression_dag_node->result_name)) + continue; + + before_window_actions->getOutputs().push_back(expression_dag_node); + before_window_actions_output_node_names.insert(expression_dag_node->result_name); + } + + expression_dag_nodes = actions_visitor.visit(before_window_actions, window_node.getPartitionByNode()); + + for (auto & expression_dag_node : expression_dag_nodes) + { + if (before_window_actions_output_node_names.contains(expression_dag_node->result_name)) + continue; + + before_window_actions->getOutputs().push_back(expression_dag_node); + before_window_actions_output_node_names.insert(expression_dag_node->result_name); + } + + /** We add only sort column sort expression in before WINDOW actions DAG. + * WITH fill expressions must be constant nodes. + */ + auto & order_by_node_list = window_node.getOrderBy(); + for (auto & sort_node : order_by_node_list.getNodes()) + { + auto & sort_node_typed = sort_node->as(); + expression_dag_nodes = actions_visitor.visit(before_window_actions, sort_node_typed.getExpression()); + + for (auto & expression_dag_node : expression_dag_nodes) + { + if (before_window_actions_output_node_names.contains(expression_dag_node->result_name)) + continue; + + before_window_actions->getOutputs().push_back(expression_dag_node); + before_window_actions_output_node_names.insert(expression_dag_node->result_name); + } + } + } + + ColumnsWithTypeAndName window_functions_additional_columns; + + for (auto & window_description : window_descriptions) + for (auto & window_function : window_description.window_functions) + window_functions_additional_columns.emplace_back(nullptr, window_function.aggregate_function->getReturnType(), window_function.column_name); + + auto before_window_step = std::make_unique(before_window_actions, + ActionsChainStep::AvailableOutputColumnsStrategy::ALL_NODES, + window_functions_additional_columns); + actions_chain.addStep(std::move(before_window_step)); + + WindowAnalysisResult result; + result.before_window_actions = std::move(before_window_actions); + result.window_descriptions = std::move(window_descriptions); + + return result; +} + +/** Construct projection analysis result. + * Projection actions are added into actions chain. + * It is client responsibility to update projection analysis result with project names actions after chain is finalized. + */ +ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, + const ColumnsWithTypeAndName & join_tree_input_columns, + const PlannerContextPtr & planner_context, + ActionsChain & actions_chain) +{ + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), projection_input, planner_context); + + auto projection_columns = query_node.getProjectionColumns(); + size_t projection_columns_size = projection_columns.size(); + + Names projection_column_names; + NamesWithAliases projection_column_names_with_display_aliases; + projection_column_names_with_display_aliases.reserve(projection_columns_size); + + auto & projection_actions_outputs = projection_actions->getOutputs(); + size_t projection_outputs_size = projection_actions_outputs.size(); + + if (projection_columns_size != projection_outputs_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "QueryTree projection nodes size mismatch. Expected {}. Actual {}", + projection_outputs_size, + projection_columns_size); + + for (size_t i = 0; i < projection_outputs_size; ++i) + { + auto & projection_column = projection_columns[i]; + const auto * projection_node = projection_actions_outputs[i]; + const auto & projection_node_name = projection_node->result_name; + + projection_column_names.push_back(projection_node_name); + projection_column_names_with_display_aliases.push_back({projection_node_name, projection_column.name}); + } + + auto projection_actions_step = std::make_unique(projection_actions); + actions_chain.addStep(std::move(projection_actions_step)); + + ProjectionAnalysisResult result; + result.projection_actions = std::move(projection_actions); + result.projection_column_names = std::move(projection_column_names); + result.projection_column_names_with_display_aliases = std::move(projection_column_names_with_display_aliases); + + return result; +} + +/** Construct sort analysis result. + * Actions before sort are added into actions chain. + */ +SortAnalysisResult analyzeSort(const QueryNode & query_node, + const ColumnsWithTypeAndName & join_tree_input_columns, + const PlannerContextPtr & planner_context, + ActionsChain & actions_chain) +{ + const auto *chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & order_by_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + + ActionsDAGPtr before_sort_actions = std::make_shared(order_by_input); + auto & before_sort_actions_outputs = before_sort_actions->getOutputs(); + before_sort_actions_outputs.clear(); + + PlannerActionsVisitor actions_visitor(planner_context); + + std::unordered_set before_sort_actions_dag_output_node_names; + + /** We add only sort node sort expression in before ORDER BY actions DAG. + * WITH fill expressions must be constant nodes. + */ + const auto & order_by_node_list = query_node.getOrderBy(); + for (const auto & sort_node : order_by_node_list.getNodes()) + { + auto & sort_node_typed = sort_node->as(); + auto expression_dag_nodes = actions_visitor.visit(before_sort_actions, sort_node_typed.getExpression()); + + for (auto & action_dag_node : expression_dag_nodes) + { + if (before_sort_actions_dag_output_node_names.contains(action_dag_node->result_name)) + continue; + + before_sort_actions_outputs.push_back(action_dag_node); + before_sort_actions_dag_output_node_names.insert(action_dag_node->result_name); + } + } + + auto actions_step_before_sort = std::make_unique(before_sort_actions); + actions_chain.addStep(std::move(actions_step_before_sort)); + + return SortAnalysisResult{std::move(before_sort_actions)}; +} + +/** Construct limit by analysis result. + * Actions before limit by are added into actions chain. + */ +LimitByAnalysisResult analyzeLimitBy(const QueryNode & query_node, + const ColumnsWithTypeAndName & join_tree_input_columns, + const PlannerContextPtr & planner_context, + ActionsChain & actions_chain) +{ + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & limit_by_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + auto before_limit_by_actions = buildActionsDAGFromExpressionNode(query_node.getLimitByNode(), limit_by_input, planner_context); + + Names limit_by_column_names; + limit_by_column_names.reserve(before_limit_by_actions->getOutputs().size()); + for (auto & output_node : before_limit_by_actions->getOutputs()) + limit_by_column_names.push_back(output_node->result_name); + + auto actions_step_before_limit_by = std::make_unique(before_limit_by_actions); + actions_chain.addStep(std::move(actions_step_before_limit_by)); + + return LimitByAnalysisResult{std::move(before_limit_by_actions), std::move(limit_by_column_names)}; +} + +} + +PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(QueryTreeNodePtr query_tree, + const ColumnsWithTypeAndName & join_tree_input_columns, + const PlannerContextPtr & planner_context) +{ + auto & query_node = query_tree->as(); + + ActionsChain actions_chain; + + std::optional where_analysis_result_optional; + std::optional where_action_step_index_optional; + + if (query_node.hasWhere()) + { + where_analysis_result_optional = analyzeFilter(query_node.getWhere(), join_tree_input_columns, planner_context, actions_chain); + where_action_step_index_optional = actions_chain.getLastStepIndex(); + } + + auto aggregation_analysis_result_optional = analyzeAggregation(query_tree, join_tree_input_columns, planner_context, actions_chain); + + std::optional having_analysis_result_optional; + std::optional having_action_step_index_optional; + + if (query_node.hasHaving()) + { + having_analysis_result_optional = analyzeFilter(query_node.getHaving(), join_tree_input_columns, planner_context, actions_chain); + having_action_step_index_optional = actions_chain.getLastStepIndex(); + } + + auto window_analysis_result_optional = analyzeWindow(query_tree, join_tree_input_columns, planner_context, actions_chain); + auto projection_analysis_result = analyzeProjection(query_node, join_tree_input_columns, planner_context, actions_chain); + + std::optional sort_analysis_result_optional; + if (query_node.hasOrderBy()) + sort_analysis_result_optional = analyzeSort(query_node, join_tree_input_columns, planner_context, actions_chain); + + std::optional limit_by_analysis_result_optional; + + if (query_node.hasLimitBy()) + limit_by_analysis_result_optional = analyzeLimitBy(query_node, join_tree_input_columns, planner_context, actions_chain); + + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + const auto & project_names_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + auto project_names_actions = std::make_shared(project_names_input); + project_names_actions->project(projection_analysis_result.projection_column_names_with_display_aliases); + actions_chain.addStep(std::make_unique(project_names_actions)); + + // std::cout << "Chain dump before finalize" << std::endl; + // std::cout << actions_chain.dump() << std::endl; + + actions_chain.finalize(); + + // std::cout << "Chain dump after finalize" << std::endl; + // std::cout << actions_chain.dump() << std::endl; + + projection_analysis_result.project_names_actions = std::move(project_names_actions); + + PlannerExpressionsAnalysisResult expressions_analysis_result(std::move(projection_analysis_result)); + + if (where_action_step_index_optional && where_analysis_result_optional) + { + auto & where_analysis_result = *where_analysis_result_optional; + auto & where_actions_chain_node = actions_chain.at(*where_action_step_index_optional); + where_analysis_result.remove_filter_column = !where_actions_chain_node->getChildRequiredOutputColumnsNames().contains(where_analysis_result.filter_column_name); + expressions_analysis_result.addWhere(std::move(where_analysis_result)); + } + + if (aggregation_analysis_result_optional) + expressions_analysis_result.addAggregation(std::move(*aggregation_analysis_result_optional)); + + if (having_action_step_index_optional && having_analysis_result_optional) + { + auto & having_analysis_result = *where_analysis_result_optional; + auto & having_actions_chain_node = actions_chain.at(*having_action_step_index_optional); + having_analysis_result.remove_filter_column = !having_actions_chain_node->getChildRequiredOutputColumnsNames().contains(having_analysis_result.filter_column_name); + expressions_analysis_result.addHaving(std::move(having_analysis_result)); + } + + if (window_analysis_result_optional) + expressions_analysis_result.addWindow(std::move(*window_analysis_result_optional)); + + if (sort_analysis_result_optional) + expressions_analysis_result.addSort(std::move(*sort_analysis_result_optional)); + + if (limit_by_analysis_result_optional) + expressions_analysis_result.addLimitBy(std::move(*limit_by_analysis_result_optional)); + + return expressions_analysis_result; +} + +} diff --git a/src/Planner/PlannerExpressionAnalysis.h b/src/Planner/PlannerExpressionAnalysis.h new file mode 100644 index 00000000000..aefb3c369d0 --- /dev/null +++ b/src/Planner/PlannerExpressionAnalysis.h @@ -0,0 +1,175 @@ +#pragma once + +#include +#include + +#include + +#include + +#include +#include +#include + +namespace DB +{ + +struct ProjectionAnalysisResult +{ + ActionsDAGPtr projection_actions; + Names projection_column_names; + NamesWithAliases projection_column_names_with_display_aliases; + ActionsDAGPtr project_names_actions; +}; + +struct FilterAnalysisResult +{ + ActionsDAGPtr filter_actions; + std::string filter_column_name; + bool remove_filter_column = false; +}; + +struct AggregationAnalysisResult +{ + ActionsDAGPtr before_aggregation_actions; + Names aggregation_keys; + AggregateDescriptions aggregate_descriptions; + GroupingSetsParamsList grouping_sets_parameters_list; + bool group_by_with_constant_keys = false; +}; + +struct WindowAnalysisResult +{ + ActionsDAGPtr before_window_actions; + std::vector window_descriptions; +}; + +struct SortAnalysisResult +{ + ActionsDAGPtr before_order_by_actions; +}; + +struct LimitByAnalysisResult +{ + ActionsDAGPtr before_limit_by_actions; + Names limit_by_column_names; +}; + +class PlannerExpressionsAnalysisResult +{ +public: + explicit PlannerExpressionsAnalysisResult(ProjectionAnalysisResult projection_analysis_result_) + : projection_analysis_result(std::move(projection_analysis_result_)) + {} + + const ProjectionAnalysisResult & getProjection() const + { + return projection_analysis_result; + } + + bool hasWhere() const + { + return where_analysis_result.filter_actions != nullptr; + } + + const FilterAnalysisResult & getWhere() const + { + return where_analysis_result; + } + + void addWhere(FilterAnalysisResult where_analysis_result_) + { + where_analysis_result = std::move(where_analysis_result_); + } + + bool hasAggregation() const + { + return !aggregation_analysis_result.aggregation_keys.empty() || !aggregation_analysis_result.aggregate_descriptions.empty(); + } + + const AggregationAnalysisResult & getAggregation() const + { + return aggregation_analysis_result; + } + + void addAggregation(AggregationAnalysisResult aggregation_analysis_result_) + { + aggregation_analysis_result = std::move(aggregation_analysis_result_); + } + + bool hasHaving() const + { + return having_analysis_result.filter_actions != nullptr; + } + + const FilterAnalysisResult & getHaving() const + { + return having_analysis_result; + } + + void addHaving(FilterAnalysisResult having_analysis_result_) + { + having_analysis_result = std::move(having_analysis_result_); + } + + bool hasWindow() const + { + return !window_analysis_result.window_descriptions.empty(); + } + + const WindowAnalysisResult & getWindow() const + { + return window_analysis_result; + } + + void addWindow(WindowAnalysisResult window_analysis_result_) + { + window_analysis_result = std::move(window_analysis_result_); + } + + bool hasSort() const + { + return sort_analysis_result.before_order_by_actions != nullptr; + } + + const SortAnalysisResult & getSort() const + { + return sort_analysis_result; + } + + void addSort(SortAnalysisResult sort_analysis_result_) + { + sort_analysis_result = std::move(sort_analysis_result_); + } + + bool hasLimitBy() const + { + return limit_by_analysis_result.before_limit_by_actions != nullptr; + } + + const LimitByAnalysisResult & getLimitBy() const + { + return limit_by_analysis_result; + } + + void addLimitBy(LimitByAnalysisResult limit_by_analysis_result_) + { + limit_by_analysis_result = std::move(limit_by_analysis_result_); + } + +private: + ProjectionAnalysisResult projection_analysis_result; + FilterAnalysisResult where_analysis_result; + AggregationAnalysisResult aggregation_analysis_result; + FilterAnalysisResult having_analysis_result; + WindowAnalysisResult window_analysis_result; + SortAnalysisResult sort_analysis_result; + LimitByAnalysisResult limit_by_analysis_result; +}; + +/// Build expression analysis result for query tree, join tree input columns and planner context +PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(QueryTreeNodePtr query_tree, + const ColumnsWithTypeAndName & join_tree_input_columns, + const PlannerContextPtr & planner_context); + +} From a0d9251f3ad27d659b2d2c79c803efb5ff0a9578 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 27 Sep 2022 17:04:03 +0200 Subject: [PATCH 107/188] Added ASOF join support --- src/Analyzer/QueryAnalysisPass.cpp | 6 +- src/Analyzer/QueryTreeBuilder.cpp | 51 ++++++++++++-- src/Analyzer/QueryTreeBuilder.h | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- .../InterpreterSelectQueryAnalyzer.cpp | 2 +- src/Planner/Planner.cpp | 2 +- src/Planner/PlannerJoinTree.cpp | 49 ++++++++++++-- src/Planner/PlannerJoins.cpp | 67 +++++++++++++------ src/Planner/PlannerJoins.h | 25 +++++++ 9 files changed, 164 insertions(+), 42 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index a6b3f159aca..a9d1dc88299 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -875,7 +875,7 @@ private: static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path); - static QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); + QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); void evaluateScalarSubquery(QueryTreeNodePtr & query_tree_node, size_t subquery_depth); @@ -994,7 +994,7 @@ QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunction(const std return {}; const auto & create_function_query = user_defined_function->as(); - auto result_node = buildQueryTree(create_function_query->function_core); + auto result_node = buildQueryTree(create_function_query->function_core, context); if (result_node->getNodeType() != QueryTreeNodeType::LAMBDA) throw Exception(ErrorCodes::LOGICAL_ERROR, "SQL user defined function {} must represent lambda expression. Actual {}", @@ -4203,7 +4203,7 @@ void QueryAnalyzer::initializeTableExpressionColumns(const QueryTreeNodePtr & ta if (column_default && column_default->kind == ColumnDefaultKind::Alias) { - auto column_node = std::make_shared(column_name_and_type, buildQueryTree(column_default->expression), table_expression_node); + auto column_node = std::make_shared(column_name_and_type, buildQueryTree(column_default->expression, context), table_expression_node); column_name_to_column_node.emplace(column_name_and_type.name, column_node); alias_columns_to_resolve.emplace_back(column_name_and_type.name, column_node); } diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 489afafdf9f..aa8fffae957 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -58,15 +58,16 @@ namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; extern const int LOGICAL_ERROR; + extern const int EXPECTED_ALL_OR_ANY; } namespace { -class QueryTreeBuilder : public WithContext +class QueryTreeBuilder { public: - explicit QueryTreeBuilder(ASTPtr query_); + explicit QueryTreeBuilder(ASTPtr query_, ContextPtr context_); QueryTreeNodePtr getQueryTreeNode() { @@ -99,12 +100,14 @@ private: ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const; ASTPtr query; + ContextPtr context; QueryTreeNodePtr query_tree_node; }; -QueryTreeBuilder::QueryTreeBuilder(ASTPtr query_) +QueryTreeBuilder::QueryTreeBuilder(ASTPtr query_, ContextPtr context_) : query(query_->clone()) + , context(std::move(context_)) { if (query->as() || query->as() || @@ -727,12 +730,46 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select else if (table_join.on_expression) join_expression = buildExpression(table_join.on_expression); + const auto & settings = context->getSettingsRef(); + auto join_default_strictness = settings.join_default_strictness; + auto any_join_distinct_right_table_keys = settings.any_join_distinct_right_table_keys; + + JoinStrictness result_join_strictness = table_join.strictness; + JoinKind result_join_kind = table_join.kind; + + if (result_join_strictness == JoinStrictness::Unspecified && (result_join_kind != JoinKind::Cross && result_join_kind != JoinKind::Comma)) + { + if (join_default_strictness == JoinStrictness::Any) + result_join_strictness = JoinStrictness::Any; + else if (join_default_strictness == JoinStrictness::All) + result_join_strictness = JoinStrictness::All; + else + throw Exception(ErrorCodes::EXPECTED_ALL_OR_ANY, + "Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty"); + } + + if (any_join_distinct_right_table_keys) + { + if (result_join_strictness == JoinStrictness::Any && result_join_kind == JoinKind::Inner) + { + result_join_strictness = JoinStrictness::Semi; + result_join_kind = JoinKind::Left; + } + + if (result_join_strictness == JoinStrictness::Any) + result_join_strictness = JoinStrictness::RightAny; + } + else if (result_join_strictness == JoinStrictness::Any && result_join_kind == JoinKind::Full) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ANY FULL JOINs are not implemented"); + } + auto join_node = std::make_shared(std::move(left_table_expression), std::move(right_table_expression), std::move(join_expression), table_join.locality, - table_join.strictness, - table_join.kind); + result_join_strictness, + result_join_kind); /** Original AST is not set because it will contain only join part and does * not include left table expression. @@ -835,9 +872,9 @@ ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & } -QueryTreeNodePtr buildQueryTree(ASTPtr query) +QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context) { - QueryTreeBuilder builder(std::move(query)); + QueryTreeBuilder builder(std::move(query), context); return builder.getQueryTreeNode(); } diff --git a/src/Analyzer/QueryTreeBuilder.h b/src/Analyzer/QueryTreeBuilder.h index ff5978c1161..3c02527436b 100644 --- a/src/Analyzer/QueryTreeBuilder.h +++ b/src/Analyzer/QueryTreeBuilder.h @@ -14,6 +14,6 @@ namespace DB * AST that represent a list of expressions ASTExpressionList. * AST that represent expression ASTIdentifier, ASTAsterisk, ASTLiteral, ASTFunction. */ -QueryTreeNodePtr buildQueryTree(ASTPtr query); +QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context); } diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 6fffa1fbfc8..f3580902c3b 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -377,7 +377,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT is supported for EXPLAIN QUERYTREE query"); auto settings = checkAndGetSettings(ast.getSettings()); - auto query_tree = buildQueryTree(ast.getExplainedQuery()); + auto query_tree = buildQueryTree(ast.getExplainedQuery(), getContext()); if (settings.run_passes) { diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 347903b84c4..bafc2cc2644 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -88,7 +88,7 @@ ASTPtr normalizeAndValidateQuery(const ASTPtr & query) QueryTreeNodePtr buildQueryTreeAndRunPasses(const ASTPtr & query, const ContextPtr & context) { - auto query_tree = buildQueryTree(query); + auto query_tree = buildQueryTree(query, context); QueryTreePassManager query_tree_pass_manager(context); addQueryTreePasses(query_tree_pass_manager); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 74c5112cc9a..8a2e98543d9 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -74,7 +74,7 @@ namespace ErrorCodes /** ClickHouse query planner. * - * TODO: JOIN support ASOF. JOIN support strictness. JOIN support constants. JOIN support ON t1.id = t1.id + * TODO: JOIN support different algorithms. JOIN support constants. JOIN support ON t1.id = t1.id * TODO: JOIN drop unnecessary columns after ON, USING section * TODO: Support display names * TODO: Support RBAC. Support RBAC for ALIAS columns diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index d5110f1a1b1..de17ecb6bde 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -35,6 +35,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INVALID_JOIN_ON_EXPRESSION; +} + namespace { @@ -150,11 +155,6 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, planner_context); auto right_plan_output_columns = right_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - if (join_node.getStrictness() == JoinStrictness::Asof) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "JOIN {} ASOF is not supported", - join_node.formatASTForErrorMessage()); - JoinClausesAndActions join_clauses_and_actions; if (join_node.isOnJoinExpression()) @@ -291,11 +291,22 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, table_join->getTableJoin() = join_node.toASTTableJoin()->as(); if (join_node.getKind() == JoinKind::Comma) table_join->getTableJoin().kind = JoinKind::Cross; - table_join->getTableJoin().strictness = JoinStrictness::All; + + table_join->getTableJoin().strictness = join_node.getStrictness(); if (join_node.isOnJoinExpression()) { const auto & join_clauses = join_clauses_and_actions.join_clauses; + bool is_asof = table_join->strictness() == JoinStrictness::Asof; + + if (join_clauses.size() > 1) + { + if (is_asof) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "ASOF join {} doesn't support multiple ORs for keys in JOIN ON section", + join_node.formatASTForErrorMessage()); + } + auto & table_join_clauses = table_join->getClauses(); for (const auto & join_clause : join_clauses) @@ -340,6 +351,32 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, const auto & join_clause_right_filter_condition_name = join_clause_get_right_filter_condition_nodes[0]->result_name; table_join_clause.analyzer_right_filter_condition_column_name = join_clause_right_filter_condition_name; } + + if (is_asof) + { + if (!join_clause.hasASOF()) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} no inequality in ASOF JOIN ON section.", + join_node.formatASTForErrorMessage()); + + if (table_join_clause.key_names_left.size() <= 1) + throw Exception(ErrorCodes::SYNTAX_ERROR, + "JOIN {} ASOF join needs at least one equi-join column", + join_node.formatASTForErrorMessage()); + } + + if (join_clause.hasASOF()) + { + const auto & asof_conditions = join_clause.getASOFConditions(); + assert(asof_conditions.size() == 1); + + const auto & asof_condition = asof_conditions[0]; + table_join->setAsofInequality(asof_condition.asof_inequality); + + /// Execution layer of JOIN algorithms expects that ASOF keys are last JOIN keys + std::swap(table_join_clause.key_names_left.at(asof_condition.key_index), table_join_clause.key_names_left.back()); + std::swap(table_join_clause.key_names_right.at(asof_condition.key_index), table_join_clause.key_names_right.back()); + } } } else if (join_node.isUsingJoinExpression()) diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index e05c83f5734..613e797655d 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -127,8 +127,13 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, const JoinNode & join_node, JoinClause & join_clause) { - /// For and function go into children - if (join_expressions_actions_node->function && join_expressions_actions_node->function->getName() == "and") + std::string function_name; + + if (join_expressions_actions_node->function) + function_name = join_expressions_actions_node->function->getName(); + + /// For 'and' function go into children + if (function_name == "and") { for (const auto & child : join_expressions_actions_node->children) { @@ -144,59 +149,77 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, return; } - if (join_expressions_actions_node->function && join_expressions_actions_node->function->getName() == "equals") + auto asof_inequality = getASOFJoinInequality(function_name); + bool is_asof_join_inequality = join_node.getStrictness() == JoinStrictness::Asof && asof_inequality != ASOFJoinInequality::None; + + if (function_name == "equals" || is_asof_join_inequality) { - const auto * equals_left_child = join_expressions_actions_node->children.at(0); - const auto * equals_right_child = join_expressions_actions_node->children.at(1); + const auto * left_child = join_expressions_actions_node->children.at(0); + const auto * right_child = join_expressions_actions_node->children.at(1); - auto left_equals_expression_side_optional = extractJoinTableSideFromExpression(equals_left_child, + auto left_expression_side_optional = extractJoinTableSideFromExpression(left_child, join_expression_dag_input_nodes, left_table_expression_columns_names, right_table_expression_columns_names, join_node); - auto right_equals_expression_side_optional = extractJoinTableSideFromExpression(equals_right_child, + auto right_expression_side_optional = extractJoinTableSideFromExpression(right_child, join_expression_dag_input_nodes, left_table_expression_columns_names, right_table_expression_columns_names, join_node); - if (!left_equals_expression_side_optional && !right_equals_expression_side_optional) + if (!left_expression_side_optional && !right_expression_side_optional) { throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "JOIN {} ON expression {} with constants is not supported", join_node.formatASTForErrorMessage(), join_expressions_actions_node->function->getName()); } - else if (left_equals_expression_side_optional && !right_equals_expression_side_optional) + else if (left_expression_side_optional && !right_expression_side_optional) { - join_clause.addCondition(*left_equals_expression_side_optional, join_expressions_actions_node); + join_clause.addCondition(*left_expression_side_optional, join_expressions_actions_node); } - else if (!left_equals_expression_side_optional && right_equals_expression_side_optional) + else if (!left_expression_side_optional && right_expression_side_optional) { - join_clause.addCondition(*right_equals_expression_side_optional, join_expressions_actions_node); + join_clause.addCondition(*right_expression_side_optional, join_expressions_actions_node); } else { - auto left_equals_expression_side = *left_equals_expression_side_optional; - auto right_equals_expression_side = *right_equals_expression_side_optional; + auto left_expression_side = *left_expression_side_optional; + auto right_expression_side = *right_expression_side_optional; - if (left_equals_expression_side != right_equals_expression_side) + if (left_expression_side != right_expression_side) { - const ActionsDAG::Node * left_key = equals_left_child; - const ActionsDAG::Node * right_key = equals_right_child; + const ActionsDAG::Node * left_key = left_child; + const ActionsDAG::Node * right_key = right_child; - if (left_equals_expression_side == JoinTableSide::Right) + if (left_expression_side_optional == JoinTableSide::Right) { - left_key = equals_right_child; - right_key = equals_left_child; + left_key = right_child; + right_key = left_child; + asof_inequality = reverseASOFJoinInequality(asof_inequality); } - join_clause.addKey(left_key, right_key); + if (is_asof_join_inequality) + { + if (join_clause.hasASOF()) + { + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} ASOF JOIN expects exactly one inequality in ON section", + join_node.formatASTForErrorMessage()); + } + + join_clause.addASOFKey(left_key, right_key, asof_inequality); + } + else + { + join_clause.addKey(left_key, right_key); + } } else { - join_clause.addCondition(left_equals_expression_side, join_expressions_actions_node); + join_clause.addCondition(left_expression_side, join_expressions_actions_node); } } diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index c8f0cb7aa42..06b4140e738 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -40,6 +40,12 @@ namespace DB class PlannerContext; using PlannerContextPtr = std::shared_ptr; +struct ASOFCondition +{ + size_t key_index; + ASOFJoinInequality asof_inequality; +}; + /// Single JOIN ON section clause representation class JoinClause { @@ -51,6 +57,13 @@ public: right_key_nodes.emplace_back(right_key_node); } + void addASOFKey(const ActionsDAG::Node * left_key_node, const ActionsDAG::Node * right_key_node, ASOFJoinInequality asof_inequality) + { + left_key_nodes.emplace_back(left_key_node); + right_key_nodes.emplace_back(right_key_node); + asof_conditions.push_back(ASOFCondition{left_key_nodes.size() - 1, asof_inequality}); + } + /// Add condition for table side void addCondition(JoinTableSide table_side, const ActionsDAG::Node * condition_node) { @@ -76,6 +89,16 @@ public: return left_key_nodes; } + bool hasASOF() const + { + return !asof_conditions.empty(); + } + + const std::vector & getASOFConditions() const + { + return asof_conditions; + } + /// Get right key nodes ActionsDAG::NodeRawConstPtrs & getRightKeyNodes() { @@ -115,6 +138,8 @@ private: ActionsDAG::NodeRawConstPtrs left_key_nodes; ActionsDAG::NodeRawConstPtrs right_key_nodes; + std::vector asof_conditions; + ActionsDAG::NodeRawConstPtrs left_filter_condition_nodes; ActionsDAG::NodeRawConstPtrs right_filter_condition_nodes; }; From 10438aea01bb209cf04d9de1d28378e0bbc8c2ff Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 28 Sep 2022 13:20:24 +0200 Subject: [PATCH 108/188] Added JOIN support for different algorithm --- src/Interpreters/DirectJoin.cpp | 15 +- src/Interpreters/DirectJoin.h | 7 + src/Interpreters/TableJoin.h | 12 ++ src/Planner/Planner.cpp | 2 +- src/Planner/PlannerJoinTree.cpp | 80 +++++++---- src/Planner/PlannerJoins.cpp | 213 ++++++++++++++++++++++++++++- src/Planner/PlannerJoins.h | 19 +++ src/Planner/TableExpressionData.h | 45 +++++- src/Storages/StorageDictionary.cpp | 6 + src/Storages/StorageDictionary.h | 4 + 10 files changed, 372 insertions(+), 31 deletions(-) diff --git a/src/Interpreters/DirectJoin.cpp b/src/Interpreters/DirectJoin.cpp index 02b3854a47b..e148db1d8e6 100644 --- a/src/Interpreters/DirectJoin.cpp +++ b/src/Interpreters/DirectJoin.cpp @@ -93,6 +93,16 @@ DirectKeyValueJoin::DirectKeyValueJoin(std::shared_ptr table_join_, LOG_TRACE(log, "Using direct join"); } +DirectKeyValueJoin::DirectKeyValueJoin( + std::shared_ptr table_join_, + const Block & right_sample_block_, + std::shared_ptr storage_, + const Block & right_sample_block_with_storage_column_names_) + : DirectKeyValueJoin(table_join_, right_sample_block_, storage_) +{ + right_sample_block_with_storage_column_names = right_sample_block_with_storage_column_names_; +} + bool DirectKeyValueJoin::addJoinedBlock(const Block &, bool) { throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unreachable code reached"); @@ -114,14 +124,15 @@ void DirectKeyValueJoin::joinBlock(Block & block, std::shared_ptr &) return; Block original_right_block = originalRightBlock(right_sample_block, *table_join); - const Names & attribute_names = original_right_block.getNames(); + Block right_block_to_use = right_sample_block_with_storage_column_names ? right_sample_block_with_storage_column_names : original_right_block; + const Names & attribute_names = right_block_to_use.getNames(); NullMap null_map; Chunk joined_chunk = storage->getByKeys({key_col}, null_map, attribute_names); /// Expected right block may differ from structure in storage, because of `join_use_nulls` or we just select not all joined attributes Block sample_storage_block = storage->getSampleBlock(attribute_names); - MutableColumns result_columns = convertBlockStructure(sample_storage_block, original_right_block, joined_chunk.mutateColumns(), null_map); + MutableColumns result_columns = convertBlockStructure(sample_storage_block, right_block_to_use, joined_chunk.mutateColumns(), null_map); for (size_t i = 0; i < result_columns.size(); ++i) { diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h index 8e82b59da02..6a6f4505474 100644 --- a/src/Interpreters/DirectJoin.h +++ b/src/Interpreters/DirectJoin.h @@ -25,6 +25,12 @@ public: const Block & right_sample_block_, std::shared_ptr storage_); + DirectKeyValueJoin( + std::shared_ptr table_join_, + const Block & right_sample_block_, + std::shared_ptr storage_, + const Block & right_sample_block_with_storage_column_names_); + virtual const TableJoin & getTableJoin() const override { return *table_join; } virtual bool addJoinedBlock(const Block &, bool) override; @@ -52,6 +58,7 @@ private: std::shared_ptr table_join; std::shared_ptr storage; Block right_sample_block; + Block right_sample_block_with_storage_column_names; Block sample_block_with_columns_to_add; Poco::Logger * log; diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 11675012d0c..316beccae80 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -165,6 +165,8 @@ private: std::string right_storage_name; + bool is_join_with_constant = false; + Names requiredJoinedNames() const; /// Create converting actions and change key column names if required @@ -279,6 +281,16 @@ public: size_t rightKeyInclusion(const String & name) const; NameSet requiredRightKeys() const; + bool isJoinWithConstant() const + { + return is_join_with_constant; + } + + void setIsJoinWithConstant(bool is_join_with_constant_value) + { + is_join_with_constant = is_join_with_constant_value; + } + bool leftBecomeNullable(const DataTypePtr & column_type) const; bool rightBecomeNullable(const DataTypePtr & column_type) const; void addJoinedColumn(const NameAndTypePair & joined_column); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 8a2e98543d9..f82d20d0ab7 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -74,7 +74,7 @@ namespace ErrorCodes /** ClickHouse query planner. * - * TODO: JOIN support different algorithms. JOIN support constants. JOIN support ON t1.id = t1.id + * TODO: Support JOIN with JOIN engine. * TODO: JOIN drop unnecessary columns after ON, USING section * TODO: Support display names * TODO: Support RBAC. Support RBAC for ALIAS columns diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index de17ecb6bde..c01d526af2c 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -156,8 +157,21 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, auto right_plan_output_columns = right_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); JoinClausesAndActions join_clauses_and_actions; + JoinKind join_kind = join_node.getKind(); - if (join_node.isOnJoinExpression()) + auto join_constant = tryExtractConstantFromJoinNode(join_tree_node); + if (join_constant) + { + /** If there is JOIN with always true constant, we transform it to cross. + * If there is JOIN with always false constant, we do not process JOIN keys. + * It is expected by join algorithm to handle such case. + * + * Example: SELECT * FROM test_table AS t1 INNER JOIN test_table AS t2 ON 1; + */ + if (*join_constant) + join_kind = JoinKind::Cross; + } + else if (join_node.isOnJoinExpression()) { join_clauses_and_actions = buildJoinClausesAndActions(left_plan_output_columns, right_plan_output_columns, @@ -250,7 +264,6 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, join_cast_plan_output_nodes(right_plan, right_plan_column_name_to_cast_type); const auto & query_context = planner_context->getQueryContext(); - JoinKind join_kind = join_node.getKind(); bool join_use_nulls = query_context->getSettingsRef().join_use_nulls; auto to_nullable_function = FunctionFactory::instance().get("toNullable", query_context); @@ -287,12 +300,14 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, } } - auto table_join = std::make_shared(); + auto table_join = std::make_shared(query_context->getSettings(), query_context->getTemporaryVolume()); table_join->getTableJoin() = join_node.toASTTableJoin()->as(); - if (join_node.getKind() == JoinKind::Comma) + table_join->getTableJoin().kind = join_kind; + + if (join_kind == JoinKind::Comma) table_join->getTableJoin().kind = JoinKind::Cross; - table_join->getTableJoin().strictness = join_node.getStrictness(); + table_join->setIsJoinWithConstant(join_constant != std::nullopt); if (join_node.isOnJoinExpression()) { @@ -420,26 +435,45 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, table_join->addJoinedColumn(column_from_joined_table); } - size_t max_block_size = query_context->getSettingsRef().max_block_size; - size_t max_streams = query_context->getSettingsRef().max_threads; - - JoinPtr join_ptr = std::make_shared(table_join, right_plan.getCurrentDataStream().header, false /*any_take_last_row*/); - QueryPlanStepPtr join_step = std::make_unique( - left_plan.getCurrentDataStream(), - right_plan.getCurrentDataStream(), - join_ptr, - max_block_size, - max_streams, - false /*optimize_read_in_order*/); - - join_step->setStepDescription(fmt::format("JOIN {}", JoinPipelineType::FillRightFirst)); - - std::vector plans; - plans.emplace_back(std::make_unique(std::move(left_plan))); - plans.emplace_back(std::make_unique(std::move(right_plan))); + auto join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), right_plan.getCurrentDataStream().header, planner_context); auto result_plan = QueryPlan(); - result_plan.unitePlans(std::move(join_step), {std::move(plans)}); + + if (join_algorithm->isFilled()) + { + size_t max_block_size = query_context->getSettingsRef().max_block_size; + + auto filled_join_step = std::make_unique( + left_plan.getCurrentDataStream(), + join_algorithm, + max_block_size); + + filled_join_step->setStepDescription("Filled JOIN"); + left_plan.addStep(std::move(filled_join_step)); + + result_plan = std::move(left_plan); + } + else + { + size_t max_block_size = query_context->getSettingsRef().max_block_size; + size_t max_streams = query_context->getSettingsRef().max_threads; + + auto join_step = std::make_unique( + left_plan.getCurrentDataStream(), + right_plan.getCurrentDataStream(), + std::move(join_algorithm), + max_block_size, + max_streams, + false /*optimize_read_in_order*/); + + join_step->setStepDescription(fmt::format("JOIN {}", JoinPipelineType::FillRightFirst)); + + std::vector plans; + plans.emplace_back(std::make_unique(std::move(left_plan))); + plans.emplace_back(std::make_unique(std::move(right_plan))); + + result_plan.unitePlans(std::move(join_step), {std::move(plans)}); + } auto drop_unused_columns_after_join_actions_dag = std::make_shared(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs updated_outputs; diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 613e797655d..91950d6a92d 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -11,13 +11,29 @@ #include #include +#include +#include +#include + #include #include #include #include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include + #include #include @@ -386,9 +402,13 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & } assert(join_clause.getLeftKeyNodes().size() == join_clause.getRightKeyNodes().size()); - size_t join_clause_left_key_nodes_size = join_clause.getLeftKeyNodes().size(); + size_t join_clause_key_nodes_size = join_clause.getLeftKeyNodes().size(); - for (size_t i = 0; i < join_clause_left_key_nodes_size; ++i) + if (join_clause_key_nodes_size == 0) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "JOIN {} cannot get JOIN keys", + join_node.formatASTForErrorMessage()); + + for (size_t i = 0; i < join_clause_key_nodes_size; ++i) { auto & left_key_node = join_clause.getLeftKeyNodes()[i]; auto & right_key_node = join_clause.getRightKeyNodes()[i]; @@ -484,4 +504,193 @@ JoinClausesAndActions buildJoinClausesAndActions( return buildJoinClausesAndActions(join_expression_input_columns, left_table_expression_columns, right_table_expression_columns, join_node_typed, planner_context); } +std::optional tryExtractConstantFromJoinNode(const QueryTreeNodePtr & join_node) +{ + auto & join_node_typed = join_node->as(); + if (!join_node_typed.getJoinExpression()) + return {}; + + auto constant_value = join_node_typed.getJoinExpression()->getConstantValueOrNull(); + if (!constant_value) + return {}; + + const auto & value = constant_value->getValue(); + auto constant_type = constant_value->getType(); + constant_type = removeNullable(removeLowCardinality(constant_type)); + + auto which_constant_type = WhichDataType(constant_type); + if (!which_constant_type.isUInt8() && !which_constant_type.isNothing()) + return {}; + + if (value.isNull()) + return false; + + UInt8 predicate_value = value.safeGet(); + return predicate_value > 0; +} + +namespace +{ + +void trySetStorageInTableJoin(const QueryTreeNodePtr & table_expression, std::shared_ptr & table_join) +{ + StoragePtr storage; + + if (auto * table_node = table_expression->as()) + storage = table_node->getStorage(); + else if (auto * table_function = table_expression->as()) + storage = table_function->getStorage(); + + if (auto storage_join = std::dynamic_pointer_cast(storage); storage_join) + { + table_join->setStorageJoin(storage_join); + return; + } + + if (!table_join->isEnabledAlgorithm(JoinAlgorithm::DIRECT)) + return; + + if (auto storage_dictionary = std::dynamic_pointer_cast(storage); storage_dictionary) + table_join->setStorageJoin(std::dynamic_pointer_cast(storage_dictionary->getDictionary())); + else if (auto storage_key_value = std::dynamic_pointer_cast(storage); storage_key_value) + table_join->setStorageJoin(storage_key_value); +} + +std::shared_ptr tryDirectJoin(const std::shared_ptr & table_join, + const QueryTreeNodePtr & right_table_expression, + const Block & right_table_expression_header, + const PlannerContextPtr & planner_context) +{ + if (!table_join->isEnabledAlgorithm(JoinAlgorithm::DIRECT)) + return {}; + + auto storage = table_join->getStorageKeyValue(); + if (!storage) + return {}; + + bool allowed_inner = isInner(table_join->kind()) && table_join->strictness() == JoinStrictness::All; + bool allowed_left = isLeft(table_join->kind()) && (table_join->strictness() == JoinStrictness::Any || + table_join->strictness() == JoinStrictness::All || + table_join->strictness() == JoinStrictness::Semi || + table_join->strictness() == JoinStrictness::Anti); + if (!allowed_inner && !allowed_left) + return {}; + + const auto & clauses = table_join->getClauses(); + bool only_one_key = clauses.size() == 1 && + clauses[0].key_names_left.size() == 1 && + clauses[0].key_names_right.size() == 1 && + !clauses[0].on_filter_condition_left && + !clauses[0].on_filter_condition_right; + + if (!only_one_key) + return {}; + + const String & key_name = clauses[0].key_names_right[0]; + + auto & right_table_expression_data = planner_context->getTableExpressionDataOrThrow(right_table_expression); + const auto * table_column_name = right_table_expression_data.getColumnNameOrNull(key_name); + if (!table_column_name) + return {}; + + const auto & storage_primary_key = storage->getPrimaryKey(); + if (storage_primary_key.size() != 1 || storage_primary_key[0] != *table_column_name) + return {}; + + /** For right table expression during execution columns have unique name. + * Direct key value join implementation during storage quering must use storage column names. + * + * Example: + * CREATE DICTIONARY test_dictionary (id UInt64, value String) PRIMARY KEY id SOURCE(CLICKHOUSE(TABLE 'test_dictionary_table')) LIFETIME(0); + * SELECT t1.id FROM test_table AS t1 INNER JOIN test_dictionary AS t2 ON t1.id = t2.id; + * + * Unique execution name for `id` column in right table expression `test_dictionary AS t2` for example can be `t2.id_0`. + * Storage column name is `id`. + * + * Here we create header for right table expression with original storage column names. + */ + Block right_table_expression_header_with_storage_column_names; + + for (const auto & right_table_expression_column : right_table_expression_header) + { + const auto * table_column_name = right_table_expression_data.getColumnNameOrNull(right_table_expression_column.name); + if (!table_column_name) + return {}; + + auto right_table_expression_column_with_storage_column_name = right_table_expression_column; + right_table_expression_column_with_storage_column_name.name = *table_column_name; + right_table_expression_header_with_storage_column_names.insert(right_table_expression_column_with_storage_column_name); + } + + return std::make_shared(table_join, right_table_expression_header, storage, right_table_expression_header_with_storage_column_names); +} + +} + +std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_join, + const QueryTreeNodePtr & right_table_expression, + const Block & right_table_expression_header, + const PlannerContextPtr & planner_context) +{ + trySetStorageInTableJoin(right_table_expression, table_join); + + /// JOIN with JOIN engine. + if (auto storage = table_join->getStorageJoin()) + return storage->getJoinLocked(table_join, planner_context->getQueryContext()); + + /** JOIN with constant. + * Example: SELECT * FROM test_table AS t1 INNER JOIN test_table AS t2 ON 1; + */ + if (table_join->isJoinWithConstant()) + { + if (!table_join->isEnabledAlgorithm(JoinAlgorithm::HASH)) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "JOIN with constant supported only with join algorithm 'hash'"); + + return std::make_shared(table_join, right_table_expression_header); + } + + if (!table_join->oneDisjunct() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH)) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Only `hash` join supports multiple ORs for keys in JOIN ON section"); + + /// Direct JOIN with special storages that support key value access. For example JOIN with Dictionary. + if (table_join->isEnabledAlgorithm(JoinAlgorithm::DIRECT)) + { + JoinPtr direct_join = tryDirectJoin(table_join, right_table_expression, right_table_expression_header, planner_context); + if (direct_join) + return direct_join; + } + + if (table_join->isEnabledAlgorithm(JoinAlgorithm::PARTIAL_MERGE) || + table_join->isEnabledAlgorithm(JoinAlgorithm::PREFER_PARTIAL_MERGE)) + { + if (MergeJoin::isSupported(table_join)) + return std::make_shared(table_join, right_table_expression_header); + } + + if (table_join->isEnabledAlgorithm(JoinAlgorithm::HASH) || + /// partial_merge is preferred, but can't be used for specified kind of join, fallback to hash + table_join->isEnabledAlgorithm(JoinAlgorithm::PREFER_PARTIAL_MERGE) || + table_join->isEnabledAlgorithm(JoinAlgorithm::PARALLEL_HASH)) + { + if (table_join->allowParallelHashJoin()) + { + auto query_context = planner_context->getQueryContext(); + return std::make_shared(query_context, table_join, query_context->getSettings().max_threads, right_table_expression_header); + } + + return std::make_shared(table_join, right_table_expression_header); + } + + if (table_join->isEnabledAlgorithm(JoinAlgorithm::FULL_SORTING_MERGE)) + { + if (FullSortingMergeJoin::isSupported(table_join)) + return std::make_shared(table_join, right_table_expression_header); + } + + if (table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO)) + return std::make_shared(table_join, right_table_expression_header); + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Can't execute any of specified algorithms for specified strictness/kind and right storage type"); +} + } diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 06b4140e738..997d7841255 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -4,6 +4,8 @@ #include #include +#include +#include #include @@ -171,4 +173,21 @@ JoinClausesAndActions buildJoinClausesAndActions( const QueryTreeNodePtr & join_node, const PlannerContextPtr & planner_context); +/** Try extract boolean constant from JOIN expression. + * Example: SELECT * FROM test_table AS t1 INNER JOIN test_table AS t2 ON 1; + * Example: SELECT * FROM test_table AS t1 INNER JOIN test_table AS t2 ON 1 != 1; + * + * join_node - join query tree node. + */ +std::optional tryExtractConstantFromJoinNode(const QueryTreeNodePtr & join_node); + +/** Choose JOIN algorithm for table join, right table expression, right table expression header and planner context. + * Table join structure can be modified during JOIN algorithm choosing for special JOIN algorithms. + * For example JOIN with Dictionary enigne, or JOIN with JOIN engine. + */ +std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_join, + const QueryTreeNodePtr & right_table_expression, + const Block & right_table_expression_header, + const PlannerContextPtr & planner_context); + } diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h index 438a3ead114..5db31a19475 100644 --- a/src/Planner/TableExpressionData.h +++ b/src/Planner/TableExpressionData.h @@ -22,6 +22,8 @@ class TableExpressionData public: using ColumnNameToColumnIdentifier = std::unordered_map; + using ColumnIdentifierToColumnName = std::unordered_map; + /// Return true if column with name exists, false otherwise bool hasColumn(const std::string & column_name) const { @@ -39,6 +41,7 @@ public: columns_names.insert(column.name); columns.push_back(column); column_name_to_column_identifier.emplace(column.name, column_identifier); + column_identifier_to_column_name.emplace(column_identifier, column.name); } /** Add column if it does not exists in table expression data. @@ -52,6 +55,7 @@ public: columns_names.insert(column.name); columns.push_back(column); column_name_to_column_identifier.emplace(column.name, column_identifier); + column_identifier_to_column_name.emplace(column_identifier, column.name); } /// Add alias column name @@ -78,14 +82,20 @@ public: return columns; } - /// Get column name to identifier map + /// Get column name to column identifier map const ColumnNameToColumnIdentifier & getColumnNameToIdentifier() const { return column_name_to_column_identifier; } + /// Get column identifier to column name map + const ColumnNameToColumnIdentifier & getColumnIdentifierToColumnName() const + { + return column_identifier_to_column_name; + } + /** Get column identifier for column name. - * Exception is thrown if there are no identifier for column name. + * Exception is thrown if there are no column identifier for column name. */ const ColumnIdentifier & getColumnIdentifierOrThrow(const std::string & column_name) const { @@ -99,7 +109,7 @@ public: } /** Get column identifier for column name. - * Null is returned if there are no identifier for column name. + * Null is returned if there are no column identifier for column name. */ const ColumnIdentifier * getColumnIdentifierOrNull(const std::string & column_name) const { @@ -110,6 +120,32 @@ public: return &it->second; } + /** Get column name for column identifier. + * Exception is thrown if there are no column name for column identifier. + */ + const std::string & getColumnNameOrThrow(const ColumnIdentifier & column_identifier) const + { + auto it = column_identifier_to_column_name.find(column_identifier); + if (it == column_identifier_to_column_name.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column name for identifier {} does not exists", + column_identifier); + + return it->second; + } + + /** Get column name for column identifier. + * Null is returned if there are no column name for column identifier. + */ + const std::string * getColumnNameOrNull(const ColumnIdentifier & column_identifier) const + { + auto it = column_identifier_to_column_name.find(column_identifier); + if (it == column_identifier_to_column_name.end()) + return nullptr; + + return &it->second; + } + /** Cache value of storage is remote method call. * * Valid only for table and table function node. @@ -138,6 +174,9 @@ private: /// Valid for table, table function, query table expression nodes ColumnNameToColumnIdentifier column_name_to_column_identifier; + /// Valid for table, table function, query table expression nodes + ColumnIdentifierToColumnName column_identifier_to_column_name; + /// Cached value if table expression receives data from remote server bool is_remote = false; }; diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index f9995cceda3..a76c4dffb5b 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -176,6 +176,12 @@ Pipe StorageDictionary::read( return dictionary->read(column_names, max_block_size, threads); } +std::shared_ptr StorageDictionary::getDictionary() const +{ + auto registered_dictionary_name = location == Location::SameDatabaseAndNameAsDictionary ? getStorageID().getInternalDictionaryName() : dictionary_name; + return getContext()->getExternalDictionariesLoader().getDictionary(registered_dictionary_name, getContext()); +} + void StorageDictionary::shutdown() { removeDictionaryConfigurationFromRepository(); diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index 6eadd1b2c21..b3442ec2f99 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -8,8 +8,10 @@ namespace DB { + struct DictionaryStructure; class TableFunctionDictionary; +class IDictionary; class StorageDictionary final : public IStorage, public WithContext { @@ -71,6 +73,8 @@ public: size_t max_block_size, size_t threads) override; + std::shared_ptr getDictionary() const; + static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure); static String generateNamesAndTypesDescription(const NamesAndTypesList & list); From d62bd792ac2d0fe0f4e247aea8ab25e4d46713d3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 28 Sep 2022 16:25:43 +0200 Subject: [PATCH 109/188] Added positional arguments support --- src/Analyzer/QueryAnalysisPass.cpp | 64 ++++++++++++++++++++++++++++++ src/Analyzer/QueryAnalysisPass.h | 2 +- 2 files changed, 65 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index a9d1dc88299..bb0e186a575 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -885,6 +885,8 @@ private: static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope); + static void replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope); + /// Resolve identifier functions QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier); @@ -1241,6 +1243,47 @@ void QueryAnalyzer::mergeWindowWithParentWindow(const QueryTreeNodePtr & window_ window_node_typed.getOrderByNode() = parent_window_node_typed.getOrderBy().clone(); } +/** Replace nodes in node list with positional arguments. + * + * Example: SELECT id, value FROM test_table GROUP BY 1, 2; + * Example: SELECT id, value FROM test_table ORDER BY 1, 2; + * Example: SELECT id, value FROM test_table LIMIT 5 BY 1, 2; + */ +void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope) +{ + auto & node_list_typed = node_list->as(); + + for (auto & node : node_list_typed.getNodes()) + { + auto * constant_node = node->as(); + if (!constant_node) + continue; + + if (!isNativeNumber(removeNullable(constant_node->getResultType()))) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Positional argument must be constant with numeric type. Actual {}. In scope {}", + constant_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + Field converted = convertFieldToType(constant_node->getValue(), DataTypeUInt64()); + if (converted.isNull()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Positional argument numeric constant expression is not representable as UInt64. In scope {}", + scope.scope_node->formatASTForErrorMessage()); + + UInt64 positional_argument_number = converted.safeGet(); + if (positional_argument_number == 0 || positional_argument_number > projection_nodes.size()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Positional argument number {} is out of bounds. Expected in range [1, {}]. In scope {}", + positional_argument_number, + projection_nodes.size(), + scope.scope_node->formatASTForErrorMessage()); + + --positional_argument_number; + node = projection_nodes[positional_argument_number]; + } +} + /// Resolve identifier functions implementation /// Try resolve table identifier from database catalog @@ -4816,10 +4859,18 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.isGroupByWithGroupingSets()) { for (auto & grouping_sets_keys_list_node : query_node_typed.getGroupBy().getNodes()) + { + if (settings.enable_positional_arguments) + replaceNodesWithPositionalArguments(grouping_sets_keys_list_node, query_node_typed.getProjection().getNodes(), scope); + resolveExpressionNodeList(grouping_sets_keys_list_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + } } else { + if (settings.enable_positional_arguments) + replaceNodesWithPositionalArguments(query_node_typed.getGroupByNode(), query_node_typed.getProjection().getNodes(), scope); + resolveExpressionNodeList(query_node_typed.getGroupByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } } @@ -4831,7 +4882,12 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier resolveWindowNodeList(query_node_typed.getWindowNode(), scope); if (query_node_typed.hasOrderBy()) + { + if (settings.enable_positional_arguments) + replaceNodesWithPositionalArguments(query_node_typed.getOrderByNode(), query_node_typed.getProjection().getNodes(), scope); + resolveSortColumnsNodeList(query_node_typed.getOrderByNode(), scope); + } if (query_node_typed.hasInterpolate()) resolveInterpolateColumnsNodeList(query_node_typed.getInterpolate(), scope); @@ -4849,7 +4905,12 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier } if (query_node_typed.hasLimitBy()) + { + if (settings.enable_positional_arguments) + replaceNodesWithPositionalArguments(query_node_typed.getLimitByNode(), query_node_typed.getProjection().getNodes(), scope); + resolveExpressionNodeList(query_node_typed.getLimitByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + } if (query_node_typed.hasLimit()) { @@ -4950,6 +5011,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier collectAggregateFunctionNodes(query_node, aggregate_function_nodes); collectWindowFunctionNodes(query_node, window_function_nodes); + if (query_node_typed.hasGroupBy()) + assertNoAggregateFunctionNodes(query_node_typed.getGroupByNode(), "in GROUP BY"); + for (auto & aggregate_function_node : aggregate_function_nodes) { auto & aggregate_function_node_typed = aggregate_function_node->as(); diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index afe73b7dd80..ed019533971 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -37,7 +37,7 @@ namespace DB * In SELECT, ORDER BY only columns that are specified in GROUP BY keys after GROUP BY are used. * GROUPING function arguments are specified in GROUP BY keys. * No GROUPING function if there is no GROUP BY. - * No aggregate functions in WHERE, PREWHERE and inside another aggregate functions. + * No aggregate functions in WHERE, PREWHERE, GROUP BY and inside another aggregate functions. * GROUP BY modifiers CUBE, ROLLUP, GROUPING SETS and WITH TOTALS. * Table expression modifiers are disabled for subqueries in JOIN TREE. * From b7d0fd6bae458561eee4f5b890e072249c3f865a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 28 Sep 2022 16:44:47 +0200 Subject: [PATCH 110/188] Added validation for subqueries and table functions in JOINs --- src/Analyzer/QueryAnalysisPass.cpp | 27 +++++++++++++++++++++++++++ src/Analyzer/QueryAnalysisPass.h | 1 + 2 files changed, 28 insertions(+) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index bb0e186a575..74accb221fc 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -105,6 +105,7 @@ namespace ErrorCodes extern const int SAMPLING_NOT_SUPPORTED; extern const int NO_COMMON_TYPE; extern const int NOT_IMPLEMENTED; + extern const int ALIAS_REQUIRED; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -879,6 +880,8 @@ private: void evaluateScalarSubquery(QueryTreeNodePtr & query_tree_node, size_t subquery_depth); + void validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + static void validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope); static void validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); @@ -1120,6 +1123,26 @@ void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node, size_t subqu union_node->performConstantFolding(std::move(constant_value)); } +void QueryAnalyzer::validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) +{ + if (!context->getSettingsRef().joined_subquery_requires_alias) + return; + + bool table_expression_has_alias = table_expression_node->hasAlias(); + if (table_expression_has_alias) + return; + + auto table_expression_node_type = table_expression_node->getNodeType(); + if (table_expression_node_type == QueryTreeNodeType::TABLE_FUNCTION || + table_expression_node_type == QueryTreeNodeType::QUERY || + table_expression_node_type == QueryTreeNodeType::UNION) + throw Exception(ErrorCodes::ALIAS_REQUIRED, + "JOIN {} no alias for subquery or table function {}. In scope {} (set joined_subquery_requires_alias = 0 to disable restriction)", + join_node->formatASTForErrorMessage(), + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); +} + void QueryAnalyzer::validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope) { const auto limit_offset_constant_value = expression_node->getConstantValueOrNull(); @@ -4415,6 +4438,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, { auto & array_join_node = join_tree_node->as(); resolveQueryJoinTreeNode(array_join_node.getTableExpression(), scope, expressions_visitor); + validateJoinTableExpressionWithoutAlias(join_tree_node, array_join_node.getTableExpression(), scope); /// Wrap array join expressions into column nodes, where array join expression is inner expression. @@ -4467,7 +4491,10 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, auto & join_node = join_tree_node->as(); resolveQueryJoinTreeNode(join_node.getLeftTableExpression(), scope, expressions_visitor); + validateJoinTableExpressionWithoutAlias(join_tree_node, join_node.getLeftTableExpression(), scope); + resolveQueryJoinTreeNode(join_node.getRightTableExpression(), scope, expressions_visitor); + validateJoinTableExpressionWithoutAlias(join_tree_node, join_node.getRightTableExpression(), scope); if (join_node.isUsingJoinExpression()) { diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index ed019533971..18482da88ce 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -40,6 +40,7 @@ namespace DB * No aggregate functions in WHERE, PREWHERE, GROUP BY and inside another aggregate functions. * GROUP BY modifiers CUBE, ROLLUP, GROUPING SETS and WITH TOTALS. * Table expression modifiers are disabled for subqueries in JOIN TREE. + * For JOIN, ARRAY JOIN subqueries and table functions must have alias (Can be changed using joined_subquery_requires_alias setting). * * 10. Special functions handling: * Function `untuple` is handled properly. From 7f26bcc27532cca0c6ba86ac882c622329626dee Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 28 Sep 2022 18:25:59 +0200 Subject: [PATCH 111/188] Added unqualified matcher support for JOIN with USING --- src/Analyzer/QueryAnalysisPass.cpp | 102 ++++++++++++++++-- src/Analyzer/Utils.cpp | 91 ++++++++++++++++ src/Analyzer/Utils.h | 18 +++- ...2382_analyzer_matcher_join_using.reference | 46 ++++++++ .../02382_analyzer_matcher_join_using.sql | 72 +++++++++++++ ...83_analyzer_merge_tree_self_join.reference | 24 +++++ .../02383_analyzer_merge_tree_self_join.sql | 44 ++++++++ 7 files changed, 386 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02382_analyzer_matcher_join_using.reference create mode 100644 tests/queries/0_stateless/02382_analyzer_matcher_join_using.sql create mode 100644 tests/queries/0_stateless/02383_analyzer_merge_tree_self_join.reference create mode 100644 tests/queries/0_stateless/02383_analyzer_merge_tree_self_join.sql diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 74accb221fc..78f047f9a6c 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -193,7 +193,6 @@ namespace ErrorCodes * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. - * TODO: JOIN support SELF JOIN with MergeTree. JOIN support matchers. * TODO: Support function identifier resolve from parent query scope, if lambda in parent scope does not capture any columns. * TODO: Support group_by_use_nulls */ @@ -2449,8 +2448,6 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( auto & matcher_node_typed = matcher_node->as(); assert(matcher_node_typed.isUnqualified()); - QueryTreeNodesWithNames matched_expression_nodes_with_column_names; - /** There can be edge case if matcher is inside lambda expression. * Try to find parent query expression using parent scopes. */ @@ -2475,15 +2472,95 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( scope.scope_node->formatASTForErrorMessage()); } - auto table_expressions = extractTableExpressions(scope_query_node->getJoinTree()); + /** For unqualifited matcher resolve we build table expressions stack from JOIN tree and then process it. + * For table, table function, query, union table expressions add matched columns into table expressions columns stack. + * For array join continue processing. + * For join node combine last left and right table expressions columns on stack together. It is important that if JOIN has USING + * we must add USING columns before combining left and right table expressions columns. Columns from left and right table + * expressions that have same names as columns in USING clause must be skipped. + */ - for (auto & table_expression : table_expressions) + auto table_expressions_stack = buildTableExpressionsStack(scope_query_node->getJoinTree()); + std::vector table_expressions_column_nodes_with_names_stack; + + for (auto & table_expression : table_expressions_stack) { + QueryTreeNodesWithNames matched_expression_nodes_with_column_names; + + if (auto * array_join_node = table_expression->as()) + continue; + + bool table_expression_in_resolve_process = scope.table_expressions_in_resolve_process.contains(table_expression.get()); + + auto * join_node = table_expression->as(); + + if (join_node) + { + size_t table_expressions_column_nodes_with_names_stack_size = table_expressions_column_nodes_with_names_stack.size(); + if (table_expressions_column_nodes_with_names_stack_size != 2) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected 2 table expressions on stack before JOIN processing. Actual {}", + table_expressions_column_nodes_with_names_stack_size); + + auto right_table_expression_columns = std::move(table_expressions_column_nodes_with_names_stack.back()); + table_expressions_column_nodes_with_names_stack.pop_back(); + + auto left_table_expression_columns = std::move(table_expressions_column_nodes_with_names_stack.back()); + table_expressions_column_nodes_with_names_stack.pop_back(); + + std::unordered_set column_names_to_skip; + + if (!table_expression_in_resolve_process && join_node->isUsingJoinExpression()) + { + auto & join_using_list = join_node->getJoinExpression()->as(); + + for (auto & join_using_node : join_using_list.getNodes()) + { + auto & column_node = join_using_node->as(); + const auto & column_name = column_node.getColumnName(); + + if (!matcher_node_typed.isMatchingColumn(column_name)) + continue; + + column_names_to_skip.insert(column_name); + + QueryTreeNodePtr column_source = getColumnSourceForJoinNodeWithUsing(table_expression); + auto matched_column_node = std::make_shared(column_node.getColumn(), column_source); + matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), column_name); + } + } + + for (auto && left_table_column : left_table_expression_columns) + { + if (column_names_to_skip.contains(left_table_column.second)) + continue; + + matched_expression_nodes_with_column_names.push_back(std::move(left_table_column)); + } + + for (auto && right_table_column : right_table_expression_columns) + { + if (column_names_to_skip.contains(right_table_column.second)) + continue; + + matched_expression_nodes_with_column_names.push_back(std::move(right_table_column)); + } + + table_expressions_column_nodes_with_names_stack.push_back(std::move(matched_expression_nodes_with_column_names)); + continue; + } + auto * table_node = table_expression->as(); auto * table_function_node = table_expression->as(); auto * query_node = table_expression->as(); auto * union_node = table_expression->as(); + if (table_expression_in_resolve_process) + { + table_expressions_column_nodes_with_names_stack.emplace_back(); + continue; + } + NamesAndTypes table_expression_columns; if (query_node || union_node) @@ -2535,9 +2612,18 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( } matcherQualifyColumnsForProjectionNamesIfNeeded(matched_expression_nodes_with_column_names, table_expression, scope); + table_expressions_column_nodes_with_names_stack.push_back(std::move(matched_expression_nodes_with_column_names)); } - return matched_expression_nodes_with_column_names; + QueryTreeNodesWithNames result; + + for (auto & table_expression_column_nodes_with_names : table_expressions_column_nodes_with_names_stack) + { + for (auto && table_expression_column_node_with_name : table_expression_column_nodes_with_names) + result.push_back(std::move(table_expression_column_node_with_name)); + } + + return result; } @@ -4444,10 +4530,6 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) { - if (array_join_expression->getNodeType() == QueryTreeNodeType::MATCHER) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "ARRAY JOIN matcher is not supported"); - auto array_join_expression_alias = array_join_expression->getAlias(); if (!array_join_expression_alias.empty() && scope.alias_name_to_expression_node.contains(array_join_expression_alias)) throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 948d7190499..394e3ae0471 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -228,4 +228,95 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node) return result; } +namespace +{ + +void buildTableExpressionsStackImpl(const QueryTreeNodePtr & join_tree_node, QueryTreeNodes & result) +{ + auto node_type = join_tree_node->getNodeType(); + + switch (node_type) + { + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::UNION: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + result.push_back(join_tree_node); + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + auto & array_join_node = join_tree_node->as(); + buildTableExpressionsStackImpl(array_join_node.getTableExpression(), result); + result.push_back(join_tree_node); + break; + } + case QueryTreeNodeType::JOIN: + { + auto & join_node = join_tree_node->as(); + buildTableExpressionsStackImpl(join_node.getLeftTableExpression(), result); + buildTableExpressionsStackImpl(join_node.getRightTableExpression(), result); + result.push_back(join_tree_node); + break; + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected node type for table expression. Expected table, table function, query, union, join or array join. Actual " + "{}", + join_tree_node->getNodeTypeName()); + } + } +} +} + +QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_node) +{ + QueryTreeNodes result; + buildTableExpressionsStackImpl(join_tree_node, result); + + return result; +} + +QueryTreeNodePtr getColumnSourceForJoinNodeWithUsing(const QueryTreeNodePtr & join_node) +{ + QueryTreeNodePtr column_source_node = join_node; + + while (true) + { + auto column_source_node_type = column_source_node->getNodeType(); + if (column_source_node_type == QueryTreeNodeType::TABLE || + column_source_node_type == QueryTreeNodeType::TABLE_FUNCTION || + column_source_node_type == QueryTreeNodeType::QUERY || + column_source_node_type == QueryTreeNodeType::UNION) + { + break; + } + else if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN) + { + auto & array_join_node = column_source_node->as(); + column_source_node = array_join_node.getTableExpression(); + continue; + } + else if (column_source_node_type == QueryTreeNodeType::JOIN) + { + auto & join_node_typed = column_source_node->as(); + column_source_node = isRight(join_node_typed.getKind()) ? join_node_typed.getRightTableExpression() : join_node_typed.getLeftTableExpression(); + continue; + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected node type for table expression. Expected table, table function, query, union, join or array join. Actual {}", + column_source_node->getNodeTypeName()); + } + } + + return column_source_node; +} + } diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 7ee0bdfcaea..359809b4bde 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -16,7 +16,23 @@ bool isNameOfInFunction(const std::string & function_name); */ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression); -/// Extract table, table function query, union from join tree +/// Extract table, table function, query, union from join tree QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node); +/** Build table expressions stack that consists from table, table function, query, union, join, array join from join tree + * Example: SELECT * FROM t1 INNER JOIN t2 INNER JOIN t3. + * Result table expressions stack: + * 1. t1 INNER JOIN t2 INNER JOIN t3 + * 2. t3 + * 3. t1 INNER JOIN t2 + * 4. t2 + * 5. t1 + */ +QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_node); + +/** Get column source for JOIN node with USING. + * Example: SELECT id FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 USING (id); + */ +QueryTreeNodePtr getColumnSourceForJoinNodeWithUsing(const QueryTreeNodePtr & join_node); + } diff --git a/tests/queries/0_stateless/02382_analyzer_matcher_join_using.reference b/tests/queries/0_stateless/02382_analyzer_matcher_join_using.reference new file mode 100644 index 00000000000..3f34634cbe8 --- /dev/null +++ b/tests/queries/0_stateless/02382_analyzer_matcher_join_using.reference @@ -0,0 +1,46 @@ +-- { echoOn } + +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) ORDER BY id, t1.value; +0 Join_1_Value_0 Join_2_Value_0 +1 Join_1_Value_1 Join_2_Value_1 +SELECT '--'; +-- +SELECT * FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) ORDER BY id, t1.value; +0 Join_1_Value_0 Join_2_Value_0 +1 Join_1_Value_1 Join_2_Value_1 +2 Join_1_Value_2 +SELECT '--'; +-- +SELECT * FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) ORDER BY id, t1.value; +0 Join_1_Value_0 Join_2_Value_0 +1 Join_1_Value_1 Join_2_Value_1 +3 Join_2_Value_3 +SELECT '--'; +-- +SELECT * FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) ORDER BY id, t1.value; +0 Join_2_Value_3 +0 Join_1_Value_0 Join_2_Value_0 +1 Join_1_Value_1 Join_2_Value_1 +2 Join_1_Value_2 +SELECT '--'; +-- +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING (id) ORDER BY id, t1.value; +0 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +1 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +SELECT '--'; +-- +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING (id) ORDER BY id, t1.value; +0 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +1 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +SELECT '--'; +-- +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING (id) ORDER BY id, t1.value; +0 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +1 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 +4 Join_3_Value_4 +SELECT '--'; +-- +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING (id) ORDER BY id, t1.value; +0 Join_3_Value_4 +0 Join_1_Value_0 Join_2_Value_0 Join_3_Value_0 +1 Join_1_Value_1 Join_2_Value_1 Join_3_Value_1 diff --git a/tests/queries/0_stateless/02382_analyzer_matcher_join_using.sql b/tests/queries/0_stateless/02382_analyzer_matcher_join_using.sql new file mode 100644 index 00000000000..29a4708aeea --- /dev/null +++ b/tests/queries/0_stateless/02382_analyzer_matcher_join_using.sql @@ -0,0 +1,72 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table_join_1; +CREATE TABLE test_table_join_1 +( + id UInt8, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_2; +CREATE TABLE test_table_join_2 +( + id UInt16, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_3; +CREATE TABLE test_table_join_3 +( + id UInt64, + value String +) ENGINE = TinyLog; + +INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value_0'); +INSERT INTO test_table_join_1 VALUES (1, 'Join_1_Value_1'); +INSERT INTO test_table_join_1 VALUES (2, 'Join_1_Value_2'); + +INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value_0'); +INSERT INTO test_table_join_2 VALUES (1, 'Join_2_Value_1'); +INSERT INTO test_table_join_2 VALUES (3, 'Join_2_Value_3'); + +INSERT INTO test_table_join_3 VALUES (0, 'Join_3_Value_0'); +INSERT INTO test_table_join_3 VALUES (1, 'Join_3_Value_1'); +INSERT INTO test_table_join_3 VALUES (4, 'Join_3_Value_4'); + +-- { echoOn } + +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) ORDER BY id, t1.value; + +SELECT '--'; + +SELECT * FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) ORDER BY id, t1.value; + +SELECT '--'; + +SELECT * FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) ORDER BY id, t1.value; + +SELECT '--'; + +SELECT * FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) ORDER BY id, t1.value; + +SELECT '--'; + +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING (id) ORDER BY id, t1.value; + +SELECT '--'; + +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING (id) ORDER BY id, t1.value; + +SELECT '--'; + +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING (id) ORDER BY id, t1.value; + +SELECT '--'; + +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING (id) ORDER BY id, t1.value; + +-- { echoOff } + +DROP TABLE test_table_join_1; +DROP TABLE test_table_join_2; +DROP TABLE test_table_join_3; diff --git a/tests/queries/0_stateless/02383_analyzer_merge_tree_self_join.reference b/tests/queries/0_stateless/02383_analyzer_merge_tree_self_join.reference new file mode 100644 index 00000000000..e48ae282f5d --- /dev/null +++ b/tests/queries/0_stateless/02383_analyzer_merge_tree_self_join.reference @@ -0,0 +1,24 @@ +-- { echoOn } + +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY t1.id, t1.value; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +SELECT '--'; +-- +SELECT * FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY t1.id, t1.value; +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_2 0 +SELECT '--'; +-- +SELECT * FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY t1.id, t1.value; +0 3 Join_2_Value_3 +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +SELECT '--'; +-- +SELECT * FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY t1.id, t1.value; +0 3 Join_2_Value_3 +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +2 Join_1_Value_2 0 diff --git a/tests/queries/0_stateless/02383_analyzer_merge_tree_self_join.sql b/tests/queries/0_stateless/02383_analyzer_merge_tree_self_join.sql new file mode 100644 index 00000000000..adb858c217f --- /dev/null +++ b/tests/queries/0_stateless/02383_analyzer_merge_tree_self_join.sql @@ -0,0 +1,44 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table_join_1; +CREATE TABLE test_table_join_1 +( + id UInt64, + value String +) ENGINE = MergeTree ORDER BY id; + +DROP TABLE IF EXISTS test_table_join_2; +CREATE TABLE test_table_join_2 +( + id UInt64, + value String +) ENGINE = MergeTree ORDER BY id; + +INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value_0'); +INSERT INTO test_table_join_1 VALUES (1, 'Join_1_Value_1'); +INSERT INTO test_table_join_1 VALUES (2, 'Join_1_Value_2'); + +INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value_0'); +INSERT INTO test_table_join_2 VALUES (1, 'Join_2_Value_1'); +INSERT INTO test_table_join_2 VALUES (3, 'Join_2_Value_3'); + +-- { echoOn } + +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY t1.id, t1.value; + +SELECT '--'; + +SELECT * FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY t1.id, t1.value; + +SELECT '--'; + +SELECT * FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY t1.id, t1.value; + +SELECT '--'; + +SELECT * FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id ORDER BY t1.id, t1.value; + +-- { echoOff } + +DROP TABLE test_table_join_1; +DROP TABLE test_table_join_2; From 3d5b20986166d92f091ff49130713c2159221efc Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 29 Sep 2022 11:16:14 +0200 Subject: [PATCH 112/188] Added support for dictGet, joinGet functions to handle identifier as first argument --- src/Analyzer/QueryAnalysisPass.cpp | 28 +++++++++ src/Analyzer/QueryAnalysisPass.h | 1 + ...02384_analyzer_dict_get_join_get.reference | 10 ++++ .../02384_analyzer_dict_get_join_get.sql | 59 +++++++++++++++++++ 4 files changed, 98 insertions(+) create mode 100644 tests/queries/0_stateless/02384_analyzer_dict_get_join_get.reference create mode 100644 tests/queries/0_stateless/02384_analyzer_dict_get_join_get.sql diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 78f047f9a6c..856d4e792fc 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -68,6 +68,7 @@ #include #include #include +#include #include #include @@ -3056,15 +3057,42 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc } bool is_special_function_in = false; + bool is_special_function_dict_get_or_join_get = false; + if (!lambda_expression_untyped) { is_special_function_in = isNameOfInFunction(function_name); + is_special_function_dict_get_or_join_get = functionIsJoinGet(function_name) || functionIsDictGet(function_name); /// Handle SELECT count(*) FROM test_table if (function_name == "count") function_node.getArguments().getNodes().clear(); } + /** Special functions dictGet and its variations and joinGet can be executed when first argument is identifier. + * Example: SELECT dictGet(identifier, 'value', toUInt64(0)); + * + * Try to resolve identifier as expression identifier and if it is resolved use it. + * Example: WITH 'dict_name' AS identifier SELECT dictGet(identifier, 'value', toUInt64(0)); + * + * Otherwise replace identifier with identifier full name constant. + * Validation that dictionary exists or table exists will be performed during function `getReturnType` method call. + */ + if (is_special_function_dict_get_or_join_get && + !function_node.getArguments().getNodes().empty() && + function_node.getArguments().getNodes()[0]->getNodeType() == QueryTreeNodeType::IDENTIFIER) + { + auto & first_argument = function_node.getArguments().getNodes()[0]; + auto & identifier_node = first_argument->as(); + IdentifierLookup identifier_lookup{identifier_node.getIdentifier(), IdentifierLookupContext::EXPRESSION}; + auto resolve_result = tryResolveIdentifier(identifier_lookup, scope); + + if (resolve_result.isResolved()) + first_argument = std::move(resolve_result.resolved_identifier); + else + first_argument = std::make_shared(identifier_node.getIdentifier().getFullName()); + } + /// Resolve function arguments resolveExpressionNodeList(function_node.getArgumentsNode(), scope, true /*allow_lambda_expression*/, is_special_function_in /*allow_table_expression*/); diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index 18482da88ce..e81589657f2 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -45,6 +45,7 @@ namespace DB * 10. Special functions handling: * Function `untuple` is handled properly. * Function `arrayJoin` is handled properly. + * For functions `dictGet` and its variations and for function `joinGet` identifier as first argument is handled properly. * * For function `grouping` arguments are resolved, but it is planner responsibility to initialize it with concrete grouping function * based on group by kind and group by keys positions. diff --git a/tests/queries/0_stateless/02384_analyzer_dict_get_join_get.reference b/tests/queries/0_stateless/02384_analyzer_dict_get_join_get.reference new file mode 100644 index 00000000000..5f783010a1c --- /dev/null +++ b/tests/queries/0_stateless/02384_analyzer_dict_get_join_get.reference @@ -0,0 +1,10 @@ +Dictionary +0 Value +Value +Value +Value +JOIN +0 Value +Value +Value +Value diff --git a/tests/queries/0_stateless/02384_analyzer_dict_get_join_get.sql b/tests/queries/0_stateless/02384_analyzer_dict_get_join_get.sql new file mode 100644 index 00000000000..79aa0703aaa --- /dev/null +++ b/tests/queries/0_stateless/02384_analyzer_dict_get_join_get.sql @@ -0,0 +1,59 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +DROP DICTIONARY IF EXISTS test_dictionary; +CREATE DICTIONARY test_dictionary +( + id UInt64, + value String +) +PRIMARY KEY id +LAYOUT(FLAT()) +SOURCE(CLICKHOUSE(TABLE 'test_table')) +LIFETIME(0); + +SELECT 'Dictionary'; + +SELECT * FROM test_dictionary; + +SELECT dictGet('test_dictionary', 'value', toUInt64(0)); + +SELECT dictGet(test_dictionary, 'value', toUInt64(0)); + +WITH 'test_dictionary' AS dictionary SELECT dictGet(dictionary, 'value', toUInt64(0)); + +WITH 'invalid_dictionary' AS dictionary SELECT dictGet(dictionary, 'value', toUInt64(0)); -- { serverError 36 } + +DROP DICTIONARY test_dictionary; +DROP TABLE test_table; + +DROP TABLE IF EXISTS test_table_join; +CREATE TABLE test_table_join +( + id UInt64, + value String +) ENGINE=Join(Any, Left, id); + +INSERT INTO test_table_join VALUES (0, 'Value'); + +SELECT 'JOIN'; + +SELECT * FROM test_table_join; + +SELECT joinGet('test_table_join', 'value', toUInt64(0)); + +SELECT joinGet(test_table_join, 'value', toUInt64(0)); + +WITH 'test_table_join' AS join_table SELECT joinGet(join_table, 'value', toUInt64(0)); + +WITH 'invalid_test_table_join' AS join_table SELECT joinGet(join_table, 'value', toUInt64(0)); -- { serverError 60 } + +DROP TABLE test_table_join; From 11324aa4b42291fdec436262bda3bb2214fac740 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 29 Sep 2022 12:11:45 +0200 Subject: [PATCH 113/188] Added IfConstantConditionPass --- src/Analyzer/IfConstantConditionPass.cpp | 68 ++++++++++++++++++++++++ src/Analyzer/IfConstantConditionPass.h | 28 ++++++++++ src/Analyzer/MultiIfToIfPass.cpp | 54 ++++++++++--------- src/Analyzer/MultiIfToIfPass.h | 1 - src/Analyzer/QueryTreePassManager.cpp | 3 ++ 5 files changed, 127 insertions(+), 27 deletions(-) create mode 100644 src/Analyzer/IfConstantConditionPass.cpp create mode 100644 src/Analyzer/IfConstantConditionPass.h diff --git a/src/Analyzer/IfConstantConditionPass.cpp b/src/Analyzer/IfConstantConditionPass.cpp new file mode 100644 index 00000000000..7aad78d78de --- /dev/null +++ b/src/Analyzer/IfConstantConditionPass.cpp @@ -0,0 +1,68 @@ +#include + +#include +#include +#include + +namespace DB +{ + +namespace +{ + +class IfConstantConditionMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + }; + + static void visit(QueryTreeNodePtr & node, Data &) + { + auto * function_node = node->as(); + if (!function_node || (function_node->getFunctionName() != "if" && function_node->getFunctionName() != "multiIf")) + return; + + if (function_node->getArguments().getNodes().size() != 3) + return; + + auto & first_argument = function_node->getArguments().getNodes()[0]; + auto first_argument_constant_value = first_argument->getConstantValueOrNull(); + if (!first_argument_constant_value) + return; + + const auto & condition_value = first_argument_constant_value->getValue(); + + bool condition_boolean_value = false; + + if (condition_value.getType() == Field::Types::Int64) + condition_boolean_value = static_cast(condition_value.safeGet()); + else if (condition_value.getType() == Field::Types::UInt64) + condition_boolean_value = static_cast(condition_value.safeGet()); + else + return; + + if (condition_boolean_value) + node = function_node->getArguments().getNodes()[1]; + else + node = function_node->getArguments().getNodes()[2]; + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) + { + return true; + } +}; + +} + +void IfConstantConditionPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) +{ + IfConstantConditionMatcher::Data data{}; + IfConstantConditionMatcher::Visitor visitor(data); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/IfConstantConditionPass.h b/src/Analyzer/IfConstantConditionPass.h new file mode 100644 index 00000000000..6fd60b450f4 --- /dev/null +++ b/src/Analyzer/IfConstantConditionPass.h @@ -0,0 +1,28 @@ +#pragma once + +#include + +namespace DB +{ + +/** Convert if with constant condition or multiIf into true condition argument value + * or false condition argument value. + * Example: SELECT if(1, true_value, false_value); + * Result: SELECT true_value; + * + * Example: SELECT if(0, true_value, false_value); + * Result: SELECT false_value; + */ +class IfConstantConditionPass : public IQueryTreePass +{ +public: + String getName() override { return "IfConstantConditionPass"; } + + String getDescription() override { return "Optimize if, multiIf for constant conditions."; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} + diff --git a/src/Analyzer/MultiIfToIfPass.cpp b/src/Analyzer/MultiIfToIfPass.cpp index 08dd64fa326..5f6aa0143aa 100644 --- a/src/Analyzer/MultiIfToIfPass.cpp +++ b/src/Analyzer/MultiIfToIfPass.cpp @@ -9,34 +9,36 @@ namespace DB namespace { - class MultiIfToIfVisitorMatcher + +class MultiIfToIfVisitorMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data { - public: - using Visitor = InDepthQueryTreeVisitor; - - struct Data - { - FunctionOverloadResolverPtr if_function_overload_resolver; - }; - - static void visit(QueryTreeNodePtr & node, Data & data) - { - auto * function_node = node->as(); - if (!function_node || function_node->getFunctionName() != "multiIf") - return; - - if (function_node->getArguments().getNodes().size() != 3) - return; - - auto result_type = function_node->getResultType(); - function_node->resolveAsFunction(data.if_function_overload_resolver, result_type); - } - - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) - { - return true; - } + FunctionOverloadResolverPtr if_function_overload_resolver; }; + + static void visit(QueryTreeNodePtr & node, Data & data) + { + auto * function_node = node->as(); + if (!function_node || function_node->getFunctionName() != "multiIf") + return; + + if (function_node->getArguments().getNodes().size() != 3) + return; + + auto result_type = function_node->getResultType(); + function_node->resolveAsFunction(data.if_function_overload_resolver, result_type); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) + { + return true; + } +}; + } void MultiIfToIfPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) diff --git a/src/Analyzer/MultiIfToIfPass.h b/src/Analyzer/MultiIfToIfPass.h index 619a4b965c7..96ac24b40f7 100644 --- a/src/Analyzer/MultiIfToIfPass.h +++ b/src/Analyzer/MultiIfToIfPass.h @@ -12,7 +12,6 @@ namespace DB class MultiIfToIfPass : public IQueryTreePass { public: - String getName() override { return "MultiIfToIf"; } String getDescription() override { return "Optimize multiIf to if for single argument."; } diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 9333fc37cb6..f436cf44c0a 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -86,6 +87,8 @@ void addQueryTreePasses(QueryTreePassManager & manager) if (settings.optimize_multiif_to_if) manager.addPass(std::make_shared()); + + manager.addPass(std::make_shared()); } } From 7c96c0afbd8066c3ccf48958136c54e72a0b6647 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 29 Sep 2022 12:43:53 +0200 Subject: [PATCH 114/188] Added OrderByTupleEliminationPass --- src/Analyzer/OrderByTupleEliminationPass.cpp | 71 ++++++++++++++++++++ src/Analyzer/OrderByTupleEliminationPass.h | 24 +++++++ src/Analyzer/QueryTreePassManager.cpp | 2 + 3 files changed, 97 insertions(+) create mode 100644 src/Analyzer/OrderByTupleEliminationPass.cpp create mode 100644 src/Analyzer/OrderByTupleEliminationPass.h diff --git a/src/Analyzer/OrderByTupleEliminationPass.cpp b/src/Analyzer/OrderByTupleEliminationPass.cpp new file mode 100644 index 00000000000..2d5d9aa3bad --- /dev/null +++ b/src/Analyzer/OrderByTupleEliminationPass.cpp @@ -0,0 +1,71 @@ +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +class OrderByTupleEliminationMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + }; + + static void visit(QueryTreeNodePtr & node, Data &) + { + auto * query_node = node->as(); + if (!query_node || !query_node->hasOrderBy()) + return; + + QueryTreeNodes result_nodes; + + for (auto & sort_node : query_node->getOrderBy().getNodes()) + { + auto & sort_node_typed = sort_node->as(); + auto * function_expression = sort_node_typed.getExpression()->as(); + if (sort_node_typed.withFill() || !function_expression || function_expression->getFunctionName() != "tuple") + { + result_nodes.push_back(sort_node); + continue; + } + + auto & tuple_arguments_nodes = function_expression->getArguments().getNodes(); + for (auto & argument_node : tuple_arguments_nodes) + { + auto result_sort_node = std::make_shared(argument_node, + sort_node_typed.getSortDirection(), + sort_node_typed.getNullsSortDirection(), + sort_node_typed.getCollator()); + result_nodes.push_back(std::move(result_sort_node)); + } + } + + query_node->getOrderBy().getNodes() = std::move(result_nodes); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) + { + return true; + } +}; + +} + +void OrderByTupleEliminationPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) +{ + OrderByTupleEliminationMatcher::Data data{}; + OrderByTupleEliminationMatcher::Visitor visitor(data); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/OrderByTupleEliminationPass.h b/src/Analyzer/OrderByTupleEliminationPass.h new file mode 100644 index 00000000000..38cd12d861f --- /dev/null +++ b/src/Analyzer/OrderByTupleEliminationPass.h @@ -0,0 +1,24 @@ +#pragma once + +#include + +namespace DB +{ + +/** Eliminate tuples from ORDER BY. + * Example: SELECT * FROM test_table ORDER BY (a, b); + * Result: SELECT * FROM test_table ORDER BY a, b; + */ +class OrderByTupleEliminationPass : public IQueryTreePass +{ +public: + String getName() override { return "OrderByTupleElimination"; } + + String getDescription() override { return "Remove tuple from ORDER BY."; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} + diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index f436cf44c0a..e829ae2c274 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -89,6 +90,7 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_shared()); manager.addPass(std::make_shared()); + manager.addPass(std::make_shared()); } } From f5ee7e0bee67869c773dc850bf8cee522579d67e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 29 Sep 2022 13:30:32 +0200 Subject: [PATCH 115/188] Added IfChainToMultiIfPass --- src/Analyzer/IfChainToMultiIfPass.cpp | 81 +++++++++++++++++++++++++++ src/Analyzer/IfChainToMultiIfPass.h | 26 +++++++++ src/Analyzer/QueryTreePassManager.cpp | 5 ++ 3 files changed, 112 insertions(+) create mode 100644 src/Analyzer/IfChainToMultiIfPass.cpp create mode 100644 src/Analyzer/IfChainToMultiIfPass.h diff --git a/src/Analyzer/IfChainToMultiIfPass.cpp b/src/Analyzer/IfChainToMultiIfPass.cpp new file mode 100644 index 00000000000..1b332e97433 --- /dev/null +++ b/src/Analyzer/IfChainToMultiIfPass.cpp @@ -0,0 +1,81 @@ +#include + +#include + +#include +#include +#include + +namespace DB +{ + +namespace +{ + +class IfChainToMultiIfPassMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + FunctionOverloadResolverPtr multi_if_function_value; + }; + + static void visit(QueryTreeNodePtr & node, Data & data) + { + auto * function_node = node->as(); + if (!function_node || function_node->getFunctionName() != "if" || function_node->getArguments().getNodes().size() != 3) + return; + + std::vector multi_if_arguments; + + auto & function_node_arguments = function_node->getArguments().getNodes(); + multi_if_arguments.insert(multi_if_arguments.end(), function_node_arguments.begin(), function_node_arguments.end()); + + QueryTreeNodePtr if_chain_node = multi_if_arguments.back(); + + while (true) + { + /// Check if last `multiIf` argument is `if` function + auto * if_chain_function_node = if_chain_node->as(); + if (!if_chain_function_node || if_chain_function_node->getFunctionName() != "if" || if_chain_function_node->getArguments().getNodes().size() != 3) + break; + + /// Replace last `multiIf` argument with `if` function arguments + + multi_if_arguments.pop_back(); + + auto & if_chain_function_node_arguments = if_chain_function_node->getArguments().getNodes(); + multi_if_arguments.insert(multi_if_arguments.end(), if_chain_function_node_arguments.begin(), if_chain_function_node_arguments.end()); + + /// Use last `multiIf` argument for next check + if_chain_node = multi_if_arguments.back(); + } + + /// Do not replace `if` with 3 arguments to `multiIf` + if (multi_if_arguments.size() <= 3) + return; + + auto multi_if_function = std::make_shared("multiIf"); + multi_if_function->resolveAsFunction(data.multi_if_function_value, std::make_shared()); + multi_if_function->getArguments().getNodes() = std::move(multi_if_arguments); + node = std::move(multi_if_function); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) + { + return true; + } +}; + +} + +void IfChainToMultiIfPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + IfChainToMultiIfPassMatcher::Data data{FunctionFactory::instance().get("multiIf", context)}; + IfChainToMultiIfPassMatcher::Visitor visitor(data); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/IfChainToMultiIfPass.h b/src/Analyzer/IfChainToMultiIfPass.h new file mode 100644 index 00000000000..b715af1d6a5 --- /dev/null +++ b/src/Analyzer/IfChainToMultiIfPass.h @@ -0,0 +1,26 @@ +#pragma once + +#include + +namespace DB +{ + +/** Convert if chain into multiIf. + * Replace if(cond_1, then_1_value, if(cond_2, ...)) chains into multiIf(cond_1, then_1_value, cond_2, ...). + * + * Example: SELECT if(cond_1, then_1_value, if(cond_2, then_2_value, else_value)); + * Result: SELECT multiIf(cond_1, then_1_value, cond_2, then_2_value, else_value); + */ +class IfChainToMultiIfPass : public IQueryTreePass +{ +public: + String getName() override { return "IfChainToMultiIfPass"; } + + String getDescription() override { return "Optimize if chain to multiIf"; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} + diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index e829ae2c274..68d7b9ac157 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -90,6 +91,10 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_shared()); manager.addPass(std::make_shared()); + + if (settings.optimize_if_chain_to_multiif) + manager.addPass(std::make_shared()); + manager.addPass(std::make_shared()); } From 54e2b6d28a014e852c1ff84828ac0d8f6ffe4a01 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 29 Sep 2022 16:31:45 +0200 Subject: [PATCH 116/188] Added EXISTS support --- src/Analyzer/QueryAnalysisPass.cpp | 51 ++++++++++++++----- src/Analyzer/QueryAnalysisPass.h | 1 + src/Planner/PlannerActionsVisitor.cpp | 13 +++-- ...76_analyzer_in_function_subquery.reference | 9 ++++ .../02376_analyzer_in_function_subquery.sql | 8 +++ 5 files changed, 64 insertions(+), 18 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 856d4e792fc..b605b20af4c 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -3017,21 +3017,21 @@ void QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const Qu */ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveScope & scope) { - FunctionNode & function_node = node->as(); - if (function_node.isResolved()) + FunctionNodePtr function_node_ptr = std::static_pointer_cast(node); + if (function_node_ptr->isResolved()) return; - const auto & function_name = function_node.getFunctionName(); + auto function_name = function_node_ptr->getFunctionName(); /// Resolve function parameters - resolveExpressionNodeList(function_node.getParametersNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + resolveExpressionNodeList(function_node_ptr->getParametersNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); /// Convert function parameters into constant parameters array Array parameters; - auto & parameters_nodes = function_node.getParameters().getNodes(); + auto & parameters_nodes = function_node_ptr->getParameters().getNodes(); parameters.reserve(parameters_nodes.size()); for (auto & parameter_node : parameters_nodes) @@ -3041,7 +3041,7 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc if (!constant_value) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for function {} expected to have constant value. Actual {}. In scope {}", - function_node.getFunctionName(), + function_name, parameter_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); @@ -3050,23 +3050,25 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc //// If function node is not window function try to lookup function node name as lambda identifier. QueryTreeNodePtr lambda_expression_untyped; - if (!function_node.isWindowFunction()) + if (!function_node_ptr->isWindowFunction()) { - auto function_lookup_result = tryResolveIdentifier({Identifier{function_node.getFunctionName()}, IdentifierLookupContext::FUNCTION}, scope); + auto function_lookup_result = tryResolveIdentifier({Identifier{function_name}, IdentifierLookupContext::FUNCTION}, scope); lambda_expression_untyped = function_lookup_result.resolved_identifier; } bool is_special_function_in = false; bool is_special_function_dict_get_or_join_get = false; + bool is_special_function_exists = false; if (!lambda_expression_untyped) { is_special_function_in = isNameOfInFunction(function_name); is_special_function_dict_get_or_join_get = functionIsJoinGet(function_name) || functionIsDictGet(function_name); + is_special_function_exists = function_name == "exists"; /// Handle SELECT count(*) FROM test_table if (function_name == "count") - function_node.getArguments().getNodes().clear(); + function_node_ptr->getArguments().getNodes().clear(); } /** Special functions dictGet and its variations and joinGet can be executed when first argument is identifier. @@ -3079,10 +3081,10 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc * Validation that dictionary exists or table exists will be performed during function `getReturnType` method call. */ if (is_special_function_dict_get_or_join_get && - !function_node.getArguments().getNodes().empty() && - function_node.getArguments().getNodes()[0]->getNodeType() == QueryTreeNodeType::IDENTIFIER) + !function_node_ptr->getArguments().getNodes().empty() && + function_node_ptr->getArguments().getNodes()[0]->getNodeType() == QueryTreeNodeType::IDENTIFIER) { - auto & first_argument = function_node.getArguments().getNodes()[0]; + auto & first_argument = function_node_ptr->getArguments().getNodes()[0]; auto & identifier_node = first_argument->as(); IdentifierLookup identifier_lookup{identifier_node.getIdentifier(), IdentifierLookupContext::EXPRESSION}; auto resolve_result = tryResolveIdentifier(identifier_lookup, scope); @@ -3095,7 +3097,30 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc /// Resolve function arguments - resolveExpressionNodeList(function_node.getArgumentsNode(), scope, true /*allow_lambda_expression*/, is_special_function_in /*allow_table_expression*/); + bool allow_table_expressions = is_special_function_in || is_special_function_exists; + resolveExpressionNodeList(function_node_ptr->getArgumentsNode(), scope, true /*allow_lambda_expression*/, allow_table_expressions /*allow_table_expression*/); + + if (is_special_function_exists) + { + /// Rewrite EXISTS (subquery) into 1 IN (SELECT 1 FROM (subquery) LIMIT 1). + auto & exists_subquery_argument = function_node_ptr->getArguments().getNodes().at(0); + + auto constant_data_type = std::make_shared(); + + auto rewritten_subquery = std::make_shared(); + rewritten_subquery->getProjection().getNodes().push_back(std::make_shared(1UL, constant_data_type)); + rewritten_subquery->getJoinTree() = exists_subquery_argument; + rewritten_subquery->getLimit() = std::make_shared(1UL, constant_data_type); + + function_node_ptr = std::make_shared("in"); + function_node_ptr->getArguments().getNodes() = {std::make_shared(1UL, constant_data_type), rewritten_subquery}; + node = function_node_ptr; + function_name = "in"; + + is_special_function_in = true; + } + + auto & function_node = *function_node_ptr; /// Replace right IN function argument if it is table or table function with subquery that read ordinary columns if (is_special_function_in) diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/QueryAnalysisPass.h index e81589657f2..231485d4bca 100644 --- a/src/Analyzer/QueryAnalysisPass.h +++ b/src/Analyzer/QueryAnalysisPass.h @@ -46,6 +46,7 @@ namespace DB * Function `untuple` is handled properly. * Function `arrayJoin` is handled properly. * For functions `dictGet` and its variations and for function `joinGet` identifier as first argument is handled properly. + * Function `exists` is converted into `in`. * * For function `grouping` arguments are resolved, but it is planner responsibility to initialize it with concrete grouping function * based on group by kind and group by keys positions. diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index d7bae6d248d..fb1badea2dd 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -347,20 +347,23 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::makeSetForInFunction(const QueryTreeNodePtr & node) { const auto & function_node = node->as(); - auto in_first_argument = function_node.getArguments().getNodes().at(0); auto in_second_argument = function_node.getArguments().getNodes().at(1); const auto & global_planner_context = planner_context->getGlobalPlannerContext(); auto set_key = global_planner_context->createSetKey(in_second_argument); auto prepared_set = global_planner_context->getSetOrThrow(set_key); - auto column_set = ColumnSet::create(1, std::move(prepared_set)); - auto column_set_const = ColumnConst::create(std::move(column_set), 1); - ColumnWithTypeAndName column; column.name = set_key; column.type = std::make_shared(); - column.column = std::move(column_set_const); + + bool set_is_created = prepared_set->isCreated(); + auto column_set = ColumnSet::create(1, std::move(prepared_set)); + + if (set_is_created) + column.column = ColumnConst::create(std::move(column_set), 1); + else + column.column = std::move(column_set); actions_stack[0].addConstantIfNecessary(set_key, column); diff --git a/tests/queries/0_stateless/02376_analyzer_in_function_subquery.reference b/tests/queries/0_stateless/02376_analyzer_in_function_subquery.reference index c0bb30b101f..3641c7d2f09 100644 --- a/tests/queries/0_stateless/02376_analyzer_in_function_subquery.reference +++ b/tests/queries/0_stateless/02376_analyzer_in_function_subquery.reference @@ -1,5 +1,14 @@ -- { echoOn } +SELECT id, value FROM test_table WHERE 1 IN (SELECT 1); +0 Value_0 +1 Value_1 +2 Value_2 +SELECT '--'; +-- +SELECT id, value FROM test_table WHERE 0 IN (SELECT 1); +SELECT '--'; +-- SELECT id, value FROM test_table WHERE id IN (SELECT 1); 1 Value_1 SELECT '--'; diff --git a/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql b/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql index 65b64f8a63a..90b2f75b6c6 100644 --- a/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql +++ b/tests/queries/0_stateless/02376_analyzer_in_function_subquery.sql @@ -19,6 +19,14 @@ INSERT INTO test_table_for_in VALUES (0), (1); -- { echoOn } +SELECT id, value FROM test_table WHERE 1 IN (SELECT 1); + +SELECT '--'; + +SELECT id, value FROM test_table WHERE 0 IN (SELECT 1); + +SELECT '--'; + SELECT id, value FROM test_table WHERE id IN (SELECT 1); SELECT '--'; From 2aff015d298a551a3e6b566e028085914614be52 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 29 Sep 2022 17:11:48 +0200 Subject: [PATCH 117/188] Added NormalizeCountVariantsPass --- src/Analyzer/IfChainToMultiIfPass.h | 2 +- src/Analyzer/IfConstantConditionPass.h | 4 +- src/Analyzer/MultiIfToIfPass.h | 2 +- src/Analyzer/NormalizeCountVariantsPass.cpp | 70 +++++++++++++++++++++ src/Analyzer/NormalizeCountVariantsPass.h | 28 +++++++++ src/Analyzer/OrderByTupleEliminationPass.h | 2 +- src/Analyzer/QueryTreePassManager.cpp | 4 ++ 7 files changed, 107 insertions(+), 5 deletions(-) create mode 100644 src/Analyzer/NormalizeCountVariantsPass.cpp create mode 100644 src/Analyzer/NormalizeCountVariantsPass.h diff --git a/src/Analyzer/IfChainToMultiIfPass.h b/src/Analyzer/IfChainToMultiIfPass.h index b715af1d6a5..6764592514a 100644 --- a/src/Analyzer/IfChainToMultiIfPass.h +++ b/src/Analyzer/IfChainToMultiIfPass.h @@ -11,7 +11,7 @@ namespace DB * Example: SELECT if(cond_1, then_1_value, if(cond_2, then_2_value, else_value)); * Result: SELECT multiIf(cond_1, then_1_value, cond_2, then_2_value, else_value); */ -class IfChainToMultiIfPass : public IQueryTreePass +class IfChainToMultiIfPass final : public IQueryTreePass { public: String getName() override { return "IfChainToMultiIfPass"; } diff --git a/src/Analyzer/IfConstantConditionPass.h b/src/Analyzer/IfConstantConditionPass.h index 6fd60b450f4..88d62c9bb48 100644 --- a/src/Analyzer/IfConstantConditionPass.h +++ b/src/Analyzer/IfConstantConditionPass.h @@ -13,12 +13,12 @@ namespace DB * Example: SELECT if(0, true_value, false_value); * Result: SELECT false_value; */ -class IfConstantConditionPass : public IQueryTreePass +class IfConstantConditionPass final : public IQueryTreePass { public: String getName() override { return "IfConstantConditionPass"; } - String getDescription() override { return "Optimize if, multiIf for constant conditions."; } + String getDescription() override { return "Optimize if, multiIf for constant condition."; } void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; diff --git a/src/Analyzer/MultiIfToIfPass.h b/src/Analyzer/MultiIfToIfPass.h index 96ac24b40f7..66fd9fd027d 100644 --- a/src/Analyzer/MultiIfToIfPass.h +++ b/src/Analyzer/MultiIfToIfPass.h @@ -9,7 +9,7 @@ namespace DB * Example: SELECT multiIf(x, 1, 0); * Result: SELECT if(x, 1, 0); */ -class MultiIfToIfPass : public IQueryTreePass +class MultiIfToIfPass final : public IQueryTreePass { public: String getName() override { return "MultiIfToIf"; } diff --git a/src/Analyzer/NormalizeCountVariantsPass.cpp b/src/Analyzer/NormalizeCountVariantsPass.cpp new file mode 100644 index 00000000000..bdba4a73ae9 --- /dev/null +++ b/src/Analyzer/NormalizeCountVariantsPass.cpp @@ -0,0 +1,70 @@ +#include + +#include +#include + +#include +#include + +namespace DB +{ + +namespace +{ + +class NormalizeCountVariantsMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + }; + + static void visit(QueryTreeNodePtr & node, Data &) + { + auto * function_node = node->as(); + if (!function_node || !function_node->isAggregateFunction() || (function_node->getFunctionName() != "count" && function_node->getFunctionName() != "sum")) + return; + + if (function_node->getArguments().getNodes().size() != 1) + return; + + auto & first_argument = function_node->getArguments().getNodes()[0]; + auto first_argument_constant_value = first_argument->getConstantValueOrNull(); + if (!first_argument_constant_value) + return; + + const auto & first_argument_constant_literal = first_argument_constant_value->getValue(); + + if (function_node->getFunctionName() == "count" && !first_argument_constant_literal.isNull()) + { + function_node->getArguments().getNodes().clear(); + } + else if (function_node->getFunctionName() == "sum" && first_argument_constant_literal.getType() == Field::Types::UInt64 && + first_argument_constant_literal.get() == 1) + { + auto result_type = function_node->getResultType(); + AggregateFunctionProperties properties; + auto aggregate_function = AggregateFunctionFactory::instance().get("count", {}, {}, properties); + function_node->resolveAsAggregateFunction(std::move(aggregate_function), std::move(result_type)); + function_node->getArguments().getNodes().clear(); + } + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) + { + return true; + } +}; + +} + +void NormalizeCountVariantsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) +{ + NormalizeCountVariantsMatcher::Data data{}; + NormalizeCountVariantsMatcher::Visitor visitor(data); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/NormalizeCountVariantsPass.h b/src/Analyzer/NormalizeCountVariantsPass.h new file mode 100644 index 00000000000..761997c5bb1 --- /dev/null +++ b/src/Analyzer/NormalizeCountVariantsPass.h @@ -0,0 +1,28 @@ +#pragma once + +#include + +namespace DB +{ + +/** Convert if with constant condition or multiIf into true condition argument value + * or false condition argument value. + * Example: SELECT count(1) + * Result: SELECT count(); + * + * Example: SELECT sum(1); + * Result: SELECT count(); + */ +class NormalizeCountVariantsPass final : public IQueryTreePass +{ +public: + String getName() override { return "NormalizeCountVariantsPass"; } + + String getDescription() override { return "Optimize count(literal), sum(1) into count()."; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} + diff --git a/src/Analyzer/OrderByTupleEliminationPass.h b/src/Analyzer/OrderByTupleEliminationPass.h index 38cd12d861f..0809953ea4a 100644 --- a/src/Analyzer/OrderByTupleEliminationPass.h +++ b/src/Analyzer/OrderByTupleEliminationPass.h @@ -9,7 +9,7 @@ namespace DB * Example: SELECT * FROM test_table ORDER BY (a, b); * Result: SELECT * FROM test_table ORDER BY a, b; */ -class OrderByTupleEliminationPass : public IQueryTreePass +class OrderByTupleEliminationPass final : public IQueryTreePass { public: String getName() override { return "OrderByTupleElimination"; } diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 68d7b9ac157..0bd6c981c19 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -87,6 +88,9 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_shared()); + if (settings.optimize_normalize_count_variants) + manager.addPass(std::make_shared()); + if (settings.optimize_multiif_to_if) manager.addPass(std::make_shared()); From b1471bdedf0f40a2bf999986e3cc9fb6ca38e452 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 29 Sep 2022 18:26:22 +0200 Subject: [PATCH 118/188] Added CountDistinctPass --- src/Analyzer/CountDistinctPass.cpp | 96 +++++++++++++++++++++++++++ src/Analyzer/CountDistinctPass.h | 28 ++++++++ src/Analyzer/QueryAnalysisPass.cpp | 11 +-- src/Analyzer/QueryTreePassManager.cpp | 32 +++++++++ 4 files changed, 162 insertions(+), 5 deletions(-) create mode 100644 src/Analyzer/CountDistinctPass.cpp create mode 100644 src/Analyzer/CountDistinctPass.h diff --git a/src/Analyzer/CountDistinctPass.cpp b/src/Analyzer/CountDistinctPass.cpp new file mode 100644 index 00000000000..69d0d7f5366 --- /dev/null +++ b/src/Analyzer/CountDistinctPass.cpp @@ -0,0 +1,96 @@ +#include + +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +class CountDistinctMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + }; + + static void visit(QueryTreeNodePtr & node, Data &) + { + auto * query_node = node->as(); + + /// Check that query has only SELECT clause + if (!query_node || (query_node->hasWith() || query_node->hasPrewhere() || query_node->hasWhere() || query_node->hasGroupBy() || + query_node->hasHaving() || query_node->hasWindow() || query_node->hasOrderBy() || query_node->hasLimitByLimit() || query_node->hasLimitByOffset() || + query_node->hasLimitBy() || query_node->hasLimit() || query_node->hasOffset())) + return; + + /// Check that query has only single table expression + auto join_tree_node_type = query_node->getJoinTree()->getNodeType(); + if (join_tree_node_type == QueryTreeNodeType::JOIN || join_tree_node_type == QueryTreeNodeType::ARRAY_JOIN) + return; + + /// Check that query has only single node in projection + auto projection_nodes = query_node->getProjection().getNodes(); + if (projection_nodes.size() != 1) + return; + + /// Check that query single projection node is `countDistinct` function + auto & projection_node = projection_nodes[0]; + auto * function_node = projection_node->as(); + if (!function_node) + return; + + if (Poco::toLower(function_node->getFunctionName()) != "countdistinct" && Poco::toLower(function_node->getFunctionName()) != "uniqexact") + return; + + /// Check that `countDistinct` function has single COLUMN argument + auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); + if (count_distinct_arguments_nodes.size() != 1 && count_distinct_arguments_nodes[0]->getNodeType() != QueryTreeNodeType::COLUMN) + return; + + auto & count_distinct_argument_column = count_distinct_arguments_nodes[0]; + auto & count_distinct_argument_column_typed = count_distinct_argument_column->as(); + + /// Build subquery SELECT count_distinct_argument_column FROM table_expression GROUP BY count_distinct_argument_column + auto subquery = std::make_shared(); + subquery->getJoinTree() = query_node->getJoinTree(); + subquery->getProjection().getNodes().push_back(count_distinct_argument_column); + subquery->getGroupBy().getNodes().push_back(count_distinct_argument_column); + subquery->resolveProjectionColumns({count_distinct_argument_column_typed.getColumn()}); + + /// Put subquery into JOIN TREE of initial query + query_node->getJoinTree() = std::move(subquery); + + /// Replace `countDistinct` of initial query into `count` + auto result_type = function_node->getResultType(); + AggregateFunctionProperties properties; + auto aggregate_function = AggregateFunctionFactory::instance().get("count", {}, {}, properties); + function_node->resolveAsAggregateFunction(std::move(aggregate_function), std::move(result_type)); + function_node->getArguments().getNodes().clear(); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) + { + return true; + } +}; + +} + +void CountDistinctPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) +{ + CountDistinctMatcher::Data data{}; + CountDistinctMatcher::Visitor visitor(data); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/CountDistinctPass.h b/src/Analyzer/CountDistinctPass.h new file mode 100644 index 00000000000..6bb0aee5e1a --- /dev/null +++ b/src/Analyzer/CountDistinctPass.h @@ -0,0 +1,28 @@ +#pragma once + +#include + +namespace DB +{ + +/** Optimize single countDistinct into count over subquery. + * + * Example: SELECT countDistinct(column) FROM table; + * Result: SELECT count() FROM (SELECT column FROM table GROUP BY column); + */ +class CountDistinctPass final : public IQueryTreePass +{ +public: + String getName() override { return "CountDistinctPass"; } + + String getDescription() override + { + return "Optimize single countDistinct into count over subquery"; + } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} + diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index b605b20af4c..91fe8ad3fcc 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -3107,13 +3107,14 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc auto constant_data_type = std::make_shared(); - auto rewritten_subquery = std::make_shared(); - rewritten_subquery->getProjection().getNodes().push_back(std::make_shared(1UL, constant_data_type)); - rewritten_subquery->getJoinTree() = exists_subquery_argument; - rewritten_subquery->getLimit() = std::make_shared(1UL, constant_data_type); + auto in_subquery = std::make_shared(); + in_subquery->getProjection().getNodes().push_back(std::make_shared(1UL, constant_data_type)); + in_subquery->getJoinTree() = exists_subquery_argument; + in_subquery->getLimit() = std::make_shared(1UL, constant_data_type); + in_subquery->resolveProjectionColumns({NameAndTypePair("1", constant_data_type)}); function_node_ptr = std::make_shared("in"); - function_node_ptr->getArguments().getNodes() = {std::make_shared(1UL, constant_data_type), rewritten_subquery}; + function_node_ptr->getArguments().getNodes() = {std::make_shared(1UL, constant_data_type), in_subquery}; node = function_node_ptr; function_name = "in"; diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 0bd6c981c19..002f723958e 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -20,6 +21,34 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +/** ClickHouse query tree pass manager + * + * TODO: Support _shard_num into shardNum() rewriting. + * TODO: Support logical expressions optimizer. + * TODO: Support fuse sum count optimize_fuse_sum_count_avg, optimize_syntax_fuse_functions. + * TODO: Support setting aggregate_functions_null_for_empty. + * TODO: Support setting optimize_functions_to_subcolumns. + * TODO: Support setting optimize_arithmetic_operations_in_aggregate_functions. + * TODO: Support setting convert_query_to_cnf. + * TODO: Support setting optimize_using_constraints. + * TODO: Support setting optimize_substitute_columns. + * TODO: Support GROUP BY injective function elimination. + * TODO: Support GROUP BY functions of other keys elimination. + * TODO: Support setting optimize_move_functions_out_of_any. + * TODO: Support setting optimize_rewrite_sum_if_to_count_if. + * TODO: Support settings.optimize_aggregators_of_group_by_keys. + * TODO: Support setting optimize_duplicate_order_by_and_distinct. + * TODO: Support setting optimize_redundant_functions_in_order_by. + * TODO: Support setting optimize_monotonous_functions_in_order_by. + * TODO: Support setting optimize_if_transform_strings_to_enum. + * TODO: Remove duplicate elements from ORDER BY clause. + * TODO: Remove duplicated elements from LIMIT BY clause. + * TODO: Remove duplicated elements from USING clause. + * TODO: Support settings.optimize_syntax_fuse_functions. + * TODO: Support settings.optimize_or_like_chain. + * TODO: Support function name normalizer. + */ + QueryTreePassManager::QueryTreePassManager(ContextPtr context_) : WithContext(context_) {} void QueryTreePassManager::addPass(QueryTreePassPtr pass) @@ -88,6 +117,9 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_shared()); + if (settings.count_distinct_optimization) + manager.addPass(std::make_shared()); + if (settings.optimize_normalize_count_variants) manager.addPass(std::make_shared()); From f2d801be2c91d6968694402ec97df031b03aec23 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 3 Oct 2022 14:22:38 +0200 Subject: [PATCH 119/188] Added CustomizeFunctionsPass --- src/Analyzer/CustomizeFunctionsPass.cpp | 173 ++++++++++++++++++++++++ src/Analyzer/CustomizeFunctionsPass.h | 26 ++++ src/Analyzer/FunctionNode.cpp | 4 +- src/Analyzer/FunctionNode.h | 8 +- src/Analyzer/QueryTreePassManager.cpp | 6 +- 5 files changed, 211 insertions(+), 6 deletions(-) create mode 100644 src/Analyzer/CustomizeFunctionsPass.cpp create mode 100644 src/Analyzer/CustomizeFunctionsPass.h diff --git a/src/Analyzer/CustomizeFunctionsPass.cpp b/src/Analyzer/CustomizeFunctionsPass.cpp new file mode 100644 index 00000000000..6a21656fb3e --- /dev/null +++ b/src/Analyzer/CustomizeFunctionsPass.cpp @@ -0,0 +1,173 @@ +#include + +#include +#include + +#include +#include +#include + +#include + +namespace DB +{ + +namespace +{ + +class CustomizeFunctionsMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + ContextPtr & context; + }; + + static void visit(QueryTreeNodePtr & node, Data & data) + { + auto * function_node = node->as(); + if (!function_node) + return; + + const auto & settings = data.context->getSettingsRef(); + + /// After successful function replacement function name and function name lowercase must be recalculated + auto function_name = function_node->getFunctionName(); + auto function_name_lowercase = Poco::toLower(function_name); + + if (function_node->isAggregateFunction()) + { + auto count_distinct_implementation_function_name = String(settings.count_distinct_implementation); + + /// Replace countDistinct with countDistinct implementation + if (function_name_lowercase == "countdistinct") + { + resolveFunctionNodeAsAggregateFunction(*function_node, count_distinct_implementation_function_name); + function_name = function_node->getFunctionName(); + function_name_lowercase = Poco::toLower(function_name); + } + + /// Replace countDistinct with countIfDistinct with countDistinctIf implementation + if (function_name_lowercase == "countifdistinct") + { + resolveFunctionNodeAsAggregateFunction(*function_node, count_distinct_implementation_function_name + "If"); + function_name = function_node->getFunctionName(); + function_name_lowercase = Poco::toLower(function_name); + } + + /// Swap aggregateFunctionIfDistinct into aggregateFunctionDistinctIf to make execution more optimal + if (function_name_lowercase.ends_with("ifdistinct")) + { + size_t prefix_length = function_name_lowercase.size() - strlen("ifdistinct"); + auto updated_function_name = function_name_lowercase.substr(0, prefix_length) + "DistinctIf"; + resolveFunctionNodeAsAggregateFunction(*function_node, updated_function_name); + function_name = function_node->getFunctionName(); + function_name_lowercase = Poco::toLower(function_name); + } + + /// Rewrite all aggregate functions to add -OrNull suffix to them + if (settings.aggregate_functions_null_for_empty && !function_name.ends_with("OrNull")) + { + auto function_properies = AggregateFunctionFactory::instance().tryGetProperties(function_name); + if (function_properies && !function_properies->returns_default_when_only_null) + { + auto updated_function_name = function_name + "OrNull"; + resolveFunctionNodeAsAggregateFunction(*function_node, updated_function_name); + function_name = function_node->getFunctionName(); + function_name_lowercase = Poco::toLower(function_name); + } + } + + /** Move -OrNull suffix ahead, this should execute after add -OrNull suffix. + * Used to rewrite aggregate functions with -OrNull suffix in some cases. + * Example: sumIfOrNull. + * Result: sumOrNullIf. + */ + if (function_name.ends_with("OrNull")) + { + auto function_properies = AggregateFunctionFactory::instance().tryGetProperties(function_name); + if (function_properies && !function_properies->returns_default_when_only_null) + { + size_t function_name_size = function_name.size(); + + static constexpr std::array suffixes_to_replace = {"MergeState", "Merge", "State", "If"}; + for (const auto & suffix : suffixes_to_replace) + { + if (!function_name_lowercase.ends_with(suffix)) + continue; + + auto updated_function_name = function_name_lowercase.substr(0, function_name_size - suffix.size()) + "OrNull" + String(suffix); + resolveFunctionNodeAsAggregateFunction(*function_node, updated_function_name); + function_name = function_node->getFunctionName(); + function_name_lowercase = Poco::toLower(function_name); + break; + } + } + } + + return; + } + + if (settings.transform_null_in) + { + auto function_result_type = function_node->getResultType(); + + static constexpr std::array, 4> in_function_to_replace_null_in_function_map = + {{ + {"in", "nullIn"}, + {"notin", "notNullIn"}, + {"globalin", "globalNullIn"}, + {"globalnotin", "globalNotNullIn"}, + }}; + + for (const auto & [in_function_name, in_function_name_to_replace] : in_function_to_replace_null_in_function_map) + { + if (function_name_lowercase == in_function_name) + { + resolveFunctionNodeAsFunction(*function_node, String(in_function_name_to_replace), data); + function_name = function_node->getFunctionName(); + function_name_lowercase = Poco::toLower(function_name); + break; + } + } + } + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) + { + return true; + } + + static inline void resolveFunctionNodeAsAggregateFunction(FunctionNode & function_node, const String & aggregate_function_name) + { + auto function_result_type = function_node.getResultType(); + auto function_aggregate_function = function_node.getAggregateFunction(); + + AggregateFunctionProperties properties; + auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, + function_aggregate_function->getArgumentTypes(), + function_aggregate_function->getParameters(), + properties); + function_node.resolveAsAggregateFunction(std::move(aggregate_function), std::move(function_result_type)); + } + + static inline void resolveFunctionNodeAsFunction(FunctionNode & function_node, const String & function_name, const Data & data) + { + auto function_result_type = function_node.getResultType(); + auto function = FunctionFactory::instance().get(function_name, data.context); + function_node.resolveAsFunction(function, std::move(function_result_type)); + } +}; + +} + +void CustomizeFunctionsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + CustomizeFunctionsMatcher::Data data{context}; + CustomizeFunctionsMatcher::Visitor visitor(data); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/CustomizeFunctionsPass.h b/src/Analyzer/CustomizeFunctionsPass.h new file mode 100644 index 00000000000..dd334db2b3f --- /dev/null +++ b/src/Analyzer/CustomizeFunctionsPass.h @@ -0,0 +1,26 @@ +#pragma once + +#include + +namespace DB +{ + +/** Customize aggregate functions and `in` functions implementations. + * + * Example: SELECT countDistinct(); + * Result: SELECT countDistinctImplementation(); + * Function countDistinctImplementation is taken from settings.count_distinct_implementation. + */ +class CustomizeFunctionsPass final : public IQueryTreePass +{ +public: + String getName() override { return "CustomizeFunctionsPass"; } + + String getDescription() override { return "Customize implementation of aggregate functions, and in functions."; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} + diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index e9d1534cb85..a5a20761556 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -119,7 +119,8 @@ bool FunctionNode::isEqualImpl(const IQueryTreeNode & rhs) const const auto & rhs_typed = assert_cast(rhs); if (function_name != rhs_typed.function_name || isAggregateFunction() != rhs_typed.isAggregateFunction() || - isNonAggregateFunction() != rhs_typed.isNonAggregateFunction()) + isOrdinaryFunction() != rhs_typed.isOrdinaryFunction() || + isWindowFunction() != rhs_typed.isWindowFunction()) return false; if (result_type && rhs_typed.result_type && !result_type->equals(*rhs_typed.getResultType())) @@ -143,6 +144,7 @@ void FunctionNode::updateTreeHashImpl(HashState & hash_state) const { hash_state.update(function_name.size()); hash_state.update(function_name); + hash_state.update(isOrdinaryFunction()); hash_state.update(isAggregateFunction()); hash_state.update(isWindowFunction()); diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index 431c896473c..519e4a15a7f 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -146,17 +146,17 @@ public: /// Is window function bool isWindowFunction() const { - return getWindowNode() != nullptr; + return isAggregateFunction() && getWindowNode() != nullptr; } /// Is function node resolved as aggregate function bool isAggregateFunction() const { - return aggregate_function != nullptr && !isWindowFunction(); + return aggregate_function != nullptr; } - /// Is function node resolved as non aggregate function - bool isNonAggregateFunction() const + /// Is function node ordinary function + bool isOrdinaryFunction() const { return function != nullptr; } diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 002f723958e..dc3eb66ea7a 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -36,7 +37,7 @@ namespace ErrorCodes * TODO: Support GROUP BY functions of other keys elimination. * TODO: Support setting optimize_move_functions_out_of_any. * TODO: Support setting optimize_rewrite_sum_if_to_count_if. - * TODO: Support settings.optimize_aggregators_of_group_by_keys. + * TODO: Support setting optimize_aggregators_of_group_by_keys. * TODO: Support setting optimize_duplicate_order_by_and_distinct. * TODO: Support setting optimize_redundant_functions_in_order_by. * TODO: Support setting optimize_monotonous_functions_in_order_by. @@ -47,6 +48,7 @@ namespace ErrorCodes * TODO: Support settings.optimize_syntax_fuse_functions. * TODO: Support settings.optimize_or_like_chain. * TODO: Support function name normalizer. + * TODO: Add optimizations based on function semantics. Example: SELECT * FROM test_table WHERE id != id. (id is not nullable column). */ QueryTreePassManager::QueryTreePassManager(ContextPtr context_) : WithContext(context_) {} @@ -123,6 +125,8 @@ void addQueryTreePasses(QueryTreePassManager & manager) if (settings.optimize_normalize_count_variants) manager.addPass(std::make_shared()); + manager.addPass(std::make_shared()); + if (settings.optimize_multiif_to_if) manager.addPass(std::make_shared()); From 0892f485bbb22a34c75a902675ab33059eff205c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 4 Oct 2022 20:43:33 +0200 Subject: [PATCH 120/188] Fix lambda tests --- src/Analyzer/LambdaNode.cpp | 1 - src/Parsers/ExpressionListParsers.cpp | 18 ++++++++++++++---- src/Planner/Planner.cpp | 7 ++++--- .../0_stateless/02343_analyzer_lambdas.sql | 4 ++-- 4 files changed, 20 insertions(+), 10 deletions(-) diff --git a/src/Analyzer/LambdaNode.cpp b/src/Analyzer/LambdaNode.cpp index 5e9f8a1861f..8f6d47e584d 100644 --- a/src/Analyzer/LambdaNode.cpp +++ b/src/Analyzer/LambdaNode.cpp @@ -80,7 +80,6 @@ ASTPtr LambdaNode::toASTImpl() const auto lambda_function_ast = std::make_shared(); lambda_function_ast->name = "lambda"; - lambda_function_ast->is_lambda_function = true; lambda_function_ast->children.push_back(std::move(lambda_function_arguments_ast)); lambda_function_ast->arguments = lambda_function_ast->children.back(); diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 1249baf1859..b4d02c4bf45 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -487,6 +487,14 @@ struct Operator std::string function_name; }; +template +static std::shared_ptr makeASTFunction(Operator & op, Args &&... args) +{ + auto ast_function = makeASTFunction(op.function_name, std::forward(args)...); + ast_function->is_lambda_function = op.type == OperatorType::Lambda; + return ast_function; +} + enum class Checkpoint { None, @@ -651,7 +659,7 @@ public: } else { - function = makeASTFunction(cur_op.function_name); + function = makeASTFunction(cur_op); if (!popLastNOperands(function->children[0]->children, cur_op.arity)) return false; @@ -2079,6 +2087,7 @@ struct ParserExpressionImpl // Recursion ParserQualifiedAsterisk qualified_asterisk_parser; ParserColumnsMatcher columns_matcher_parser; + ParserQualifiedColumnsMatcher qualified_columns_matcher_parser; ParserSubquery subquery_parser; bool parse(std::unique_ptr start, IParser::Pos & pos, ASTPtr & node, Expected & expected); @@ -2307,7 +2316,7 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos if (!layers.back()->popOperand(argument)) return Action::NONE; - function = makeASTFunction(prev_op.function_name, argument, tmp); + function = makeASTFunction(prev_op, argument, tmp); if (!modifyAST(function, subquery_function_type)) return Action::NONE; @@ -2356,7 +2365,8 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos literal_parser.parse(pos, tmp, expected) || asterisk_parser.parse(pos, tmp, expected) || qualified_asterisk_parser.parse(pos, tmp, expected) || - columns_matcher_parser.parse(pos, tmp, expected)) + columns_matcher_parser.parse(pos, tmp, expected) || + qualified_columns_matcher_parser.parse(pos, tmp, expected)) { layers.back()->pushOperand(std::move(tmp)); } @@ -2491,7 +2501,7 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po } else { - function = makeASTFunction(prev_op.function_name); + function = makeASTFunction(prev_op); if (!layers.back()->popLastNOperands(function->children[0]->children, prev_op.arity)) return Action::NONE; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index f82d20d0ab7..1597c7a5e95 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -343,12 +343,13 @@ void Planner::buildQueryPlanIfNeeded() settings.empty_result_for_aggregation_by_empty_set || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && aggregation_analysis_result.aggregation_keys.empty() && aggregation_analysis_result.group_by_with_constant_keys), - planner_context->getQueryContext()->getTemporaryVolume(), + planner_context->getQueryContext()->getTempDataOnDisk(), settings.max_threads, settings.min_free_disk_space_for_temporary_data, settings.compile_aggregate_expressions, settings.min_count_to_compile_aggregate_expression, settings.max_block_size, + settings.enable_software_prefetch_in_aggregation, /* only_merge */ false, stats_collecting_params ); @@ -478,7 +479,7 @@ void Planner::buildQueryPlanIfNeeded() settings.max_bytes_before_remerge_sort, settings.remerge_sort_lowered_memory_bytes_ratio, settings.max_bytes_before_external_sort, - query_context->getTemporaryVolume(), + query_context->getTempDataOnDisk(), settings.min_free_disk_space_for_temporary_data, settings.optimize_sorting_by_input_stream_properties); @@ -550,7 +551,7 @@ void Planner::buildQueryPlanIfNeeded() settings.max_bytes_before_remerge_sort, settings.remerge_sort_lowered_memory_bytes_ratio, settings.max_bytes_before_external_sort, - planner_context->getQueryContext()->getTemporaryVolume(), + planner_context->getQueryContext()->getTempDataOnDisk(), settings.min_free_disk_space_for_temporary_data, settings.optimize_sorting_by_input_stream_properties); diff --git a/tests/queries/0_stateless/02343_analyzer_lambdas.sql b/tests/queries/0_stateless/02343_analyzer_lambdas.sql index 97e86b0ee39..1c6200e5eb6 100644 --- a/tests/queries/0_stateless/02343_analyzer_lambdas.sql +++ b/tests/queries/0_stateless/02343_analyzer_lambdas.sql @@ -62,8 +62,8 @@ WITH x -> untuple(x) AS lambda SELECT cast((1, 'Value'), 'Tuple (id UInt64, valu SELECT 'Lambda carrying'; -WITH functor, x -> functor(x) AS lambda, x -> x + 1 AS functor_1, x -> toString(x) AS functor_2 SELECT lambda(functor_1, 1), lambda(functor_2, 1); -WITH functor, x -> functor(x) AS lambda, x -> x + 1 AS functor_1, x -> toString(x) AS functor_2 SELECT lambda(functor_1, id), lambda(functor_2, id) FROM test_table; +WITH (functor, x) -> functor(x) AS lambda, x -> x + 1 AS functor_1, x -> toString(x) AS functor_2 SELECT lambda(functor_1, 1), lambda(functor_2, 1); +WITH (functor, x) -> functor(x) AS lambda, x -> x + 1 AS functor_1, x -> toString(x) AS functor_2 SELECT lambda(functor_1, id), lambda(functor_2, id) FROM test_table; DROP TABLE test_table_tuple; DROP TABLE test_table; From 86896b76c54cd564d302325df29afb7761b382c3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 4 Oct 2022 22:34:22 +0200 Subject: [PATCH 121/188] Fix GROUP BY validation for subqueries --- src/Analyzer/QueryAnalysisPass.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 91fe8ad3fcc..7b03a6fee3c 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -4732,6 +4732,7 @@ public: struct Data { const QueryTreeNodes & group_by_keys_nodes; + const IdentifierResolveScope & scope; }; static void visit(const QueryTreeNodePtr & node, Data & data) @@ -4761,8 +4762,9 @@ public: if (!found_argument_in_group_by_keys) throw Exception(ErrorCodes::NOT_AN_AGGREGATE, - "GROUPING function argument {} is not in GROUP BY", - grouping_function_arguments_node->formatASTForErrorMessage()); + "GROUPING function argument {} is not in GROUP BY. In scope {}", + grouping_function_arguments_node->formatASTForErrorMessage(), + data.scope.scope_node->formatASTForErrorMessage()); } return; @@ -4792,8 +4794,9 @@ public: column_name += '.' + column_node->getColumnName(); throw Exception(ErrorCodes::NOT_AN_AGGREGATE, - "Column {} is not under aggregate function and not in GROUP BY", - column_name); + "Column {} is not under aggregate function and not in GROUP BY. In scope {}", + column_name, + data.scope.scope_node->formatASTForErrorMessage()); } static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node, Data & data) @@ -4811,7 +4814,7 @@ public: } } - return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; + return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); } }; @@ -5235,7 +5238,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (has_aggregation) { - ValidateGroupByColumnsVisitor::Data validate_group_by_visitor_data {group_by_keys_nodes}; + ValidateGroupByColumnsVisitor::Data validate_group_by_visitor_data {group_by_keys_nodes, scope}; ValidateGroupByColumnsVisitor validate_group_by_visitor(validate_group_by_visitor_data); if (query_node_typed.hasHaving()) From 400de59a37a70a8495a0aa8d88b13f415ce66bc8 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 4 Oct 2022 22:45:47 +0200 Subject: [PATCH 122/188] Fixed style check --- src/Analyzer/QueryAnalysisPass.cpp | 1 + src/Analyzer/QueryTreeBuilder.cpp | 1 + src/Planner/PlannerExpressionAnalysis.cpp | 5 +++++ src/Planner/PlannerJoinTree.cpp | 3 +++ src/Planner/PlannerJoins.cpp | 4 ++-- 5 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 7b03a6fee3c..90da0dbba6e 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -107,6 +107,7 @@ namespace ErrorCodes extern const int NO_COMMON_TYPE; extern const int NOT_IMPLEMENTED; extern const int ALIAS_REQUIRED; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index aa8fffae957..30d00bc5e2f 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -59,6 +59,7 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; extern const int LOGICAL_ERROR; extern const int EXPECTED_ALL_OR_ANY; + extern const int NOT_IMPLEMENTED; } namespace diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index f428668baef..3b85bc68e85 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -19,6 +19,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index c01d526af2c..4be2d0762a7 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -39,6 +39,9 @@ namespace DB namespace ErrorCodes { extern const int INVALID_JOIN_ON_EXPRESSION; + extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; + extern const int SYNTAX_ERROR; } namespace diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 91950d6a92d..59c6e47a50d 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -44,7 +44,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int INVALID_JOIN_ON_EXPRESSION; - + extern const int NOT_IMPLEMENTED; } void JoinClause::dump(WriteBuffer & buffer) const @@ -598,7 +598,7 @@ std::shared_ptr tryDirectJoin(const std::shared_ptr Date: Tue, 4 Oct 2022 22:46:24 +0200 Subject: [PATCH 123/188] Fixed tests --- .../02366_explain_query_tree.reference | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/02366_explain_query_tree.reference b/tests/queries/0_stateless/02366_explain_query_tree.reference index d2a0e875621..36cb367d946 100644 --- a/tests/queries/0_stateless/02366_explain_query_tree.reference +++ b/tests/queries/0_stateless/02366_explain_query_tree.reference @@ -1,11 +1,11 @@ -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary PROJECTION LIST id: 1, nodes: 1 CONSTANT id: 2, constant_value: UInt64_1, constant_value_type: UInt8 JOIN TREE IDENTIFIER id: 3, identifier: system.one -- -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary PROJECTION LIST id: 1, nodes: 2 IDENTIFIER id: 2, identifier: id @@ -13,7 +13,7 @@ QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, i JOIN TREE IDENTIFIER id: 4, identifier: test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary PROJECTION COLUMNS id UInt64 value String @@ -24,10 +24,10 @@ QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, i JOIN TREE TABLE id: 3, table_name: default.test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary PROJECTION LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: arrayMap, is_aggregate_function: 0 + FUNCTION id: 2, function_name: arrayMap, function_type: ordinary ARGUMENTS LIST id: 3, nodes: 2 LAMBDA id: 4 @@ -35,7 +35,7 @@ QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, i LIST id: 5, nodes: 1 IDENTIFIER id: 6, identifier: x EXPRESSION - FUNCTION id: 7, function_name: plus, is_aggregate_function: 0 + FUNCTION id: 7, function_name: plus, function_type: ordinary ARGUMENTS LIST id: 8, nodes: 2 IDENTIFIER id: 9, identifier: x @@ -44,12 +44,12 @@ QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, i JOIN TREE IDENTIFIER id: 12, identifier: test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary PROJECTION COLUMNS arrayMap(lambda(tuple(x), plus(x, 1)), [1, 2, 3]) Array(UInt16) PROJECTION LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: arrayMap, is_aggregate_function: 0, result_type: Array(UInt16) + FUNCTION id: 2, function_name: arrayMap, function_type: ordinary, result_type: Array(UInt16) ARGUMENTS LIST id: 3, nodes: 2 LAMBDA id: 4 @@ -57,7 +57,7 @@ QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, i LIST id: 5, nodes: 1 COLUMN id: 6, column_name: x, result_type: UInt8, source_id: 4 EXPRESSION - FUNCTION id: 7, function_name: plus, is_aggregate_function: 0, result_type: UInt16 + FUNCTION id: 7, function_name: plus, function_type: ordinary, result_type: UInt16 ARGUMENTS LIST id: 8, nodes: 2 COLUMN id: 6, column_name: x, result_type: UInt8, source_id: 4 @@ -66,7 +66,7 @@ QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, i JOIN TREE TABLE id: 11, table_name: default.test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary WITH LIST id: 1, nodes: 1 LAMBDA id: 2, alias: lambda @@ -74,26 +74,26 @@ QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, i LIST id: 3, nodes: 1 IDENTIFIER id: 4, identifier: x EXPRESSION - FUNCTION id: 5, function_name: plus, is_aggregate_function: 0 + FUNCTION id: 5, function_name: plus, function_type: ordinary ARGUMENTS LIST id: 6, nodes: 2 IDENTIFIER id: 7, identifier: x CONSTANT id: 8, constant_value: UInt64_1, constant_value_type: UInt8 PROJECTION LIST id: 9, nodes: 1 - FUNCTION id: 10, function_name: lambda, is_aggregate_function: 0 + FUNCTION id: 10, function_name: lambda, function_type: ordinary ARGUMENTS LIST id: 11, nodes: 1 IDENTIFIER id: 12, identifier: id JOIN TREE IDENTIFIER id: 13, identifier: test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, is_group_by_with_rollup: 0, is_group_by_with_cube: 0, is_group_by_with_grouping_sets: 0 +QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary PROJECTION COLUMNS lambda(id) UInt64 PROJECTION LIST id: 1, nodes: 1 - FUNCTION id: 2, function_name: plus, is_aggregate_function: 0, result_type: UInt64 + FUNCTION id: 2, function_name: plus, function_type: ordinary, result_type: UInt64 ARGUMENTS LIST id: 3, nodes: 2 COLUMN id: 4, column_name: id, result_type: UInt64, source_id: 5 From 048e776c48c4dc3262f8a14692f728c7c1f44053 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 5 Oct 2022 12:09:23 +0200 Subject: [PATCH 124/188] Fixed tests --- src/Analyzer/QueryAnalysisPass.cpp | 4 ++-- src/Analyzer/QueryTreeBuilder.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 90da0dbba6e..885ca88ba4f 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -901,7 +901,7 @@ private: QueryTreeNodePtr tryResolveIdentifierFromTableColumns(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); - bool tryBindIdentifierToTableExpression(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + static bool tryBindIdentifierToTableExpression(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); QueryTreeNodePtr tryResolveIdentifierFromTableExpression(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); @@ -921,7 +921,7 @@ private: using QueryTreeNodesWithNames = std::vector>; - void matcherQualifyColumnsForProjectionNamesIfNeeded(QueryTreeNodesWithNames & matched_nodes_with_column_names, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + static void matcherQualifyColumnsForProjectionNamesIfNeeded(QueryTreeNodesWithNames & matched_nodes_with_column_names, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); QueryTreeNodesWithNames resolveQualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 30d00bc5e2f..531fe0ae5d8 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -440,7 +440,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co const auto * lambda_argument_identifier = lambda_argument->as(); if (!lambda_argument_identifier) - throw Exception(ErrorCodes::LOGICAL_ERROR, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Lambda {} argument is not identifier", function->formatForErrorMessage()); From 239f1855aa2515214516fd3c868bf7b1c120b804 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 6 Oct 2022 21:36:06 +0200 Subject: [PATCH 125/188] Updated projection names implementation --- src/Analyzer/ColumnTransformers.cpp | 3 +- src/Analyzer/ColumnTransformers.h | 2 - src/Analyzer/ConstantNode.cpp | 8 +- src/Analyzer/ConstantNode.h | 10 +- src/Analyzer/FunctionNode.h | 8 +- src/Analyzer/QueryAnalysisPass.cpp | 1455 ++++++++++------- src/Analyzer/QueryNode.cpp | 24 +- src/Analyzer/QueryTreeBuilder.cpp | 1 + src/Analyzer/WindowNode.cpp | 4 +- src/Planner/CollectSets.cpp | 2 +- src/Planner/Planner.cpp | 5 +- .../02378_analyzer_projection_names.reference | 226 ++- .../02378_analyzer_projection_names.sql | 177 +- 13 files changed, 1341 insertions(+), 584 deletions(-) diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp index 9bcc1bb2d02..6568d82afc0 100644 --- a/src/Analyzer/ColumnTransformers.cpp +++ b/src/Analyzer/ColumnTransformers.cpp @@ -106,8 +106,7 @@ ASTPtr ApplyColumnTransformerNode::toASTImpl() const QueryTreeNodePtr ApplyColumnTransformerNode::cloneImpl() const { - ApplyColumnTransformerNodePtr result_apply_transformer(new ApplyColumnTransformerNode()); - return result_apply_transformer; + return std::make_shared(getExpressionNode()); } /// ExceptColumnTransformerNode implementation diff --git a/src/Analyzer/ColumnTransformers.h b/src/Analyzer/ColumnTransformers.h index f32ec1b96cf..f788b6fe5e6 100644 --- a/src/Analyzer/ColumnTransformers.h +++ b/src/Analyzer/ColumnTransformers.h @@ -141,8 +141,6 @@ protected: QueryTreeNodePtr cloneImpl() const override; private: - ApplyColumnTransformerNode() = default; - ApplyColumnTransformerType apply_transformer_type = ApplyColumnTransformerType::LAMBDA; static constexpr size_t expression_child_index = 0; }; diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 380c4aba2a1..9539f9fa142 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -18,7 +18,7 @@ namespace DB ConstantNode::ConstantNode(ConstantValuePtr constant_value_) : constant_value(std::move(constant_value_)) - , value_string_dump(applyVisitor(FieldVisitorToString(), constant_value->getValue())) + , value_string(applyVisitor(FieldVisitorToString(), constant_value->getValue())) { } @@ -44,7 +44,7 @@ void ConstantNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state bool ConstantNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); - return *constant_value == *rhs_typed.constant_value && value_string_dump == rhs_typed.value_string_dump; + return *constant_value == *rhs_typed.constant_value && value_string == rhs_typed.value_string; } void ConstantNode::updateTreeHashImpl(HashState & hash_state) const @@ -53,8 +53,8 @@ void ConstantNode::updateTreeHashImpl(HashState & hash_state) const hash_state.update(type_name.size()); hash_state.update(type_name); - hash_state.update(value_string_dump.size()); - hash_state.update(value_string_dump); + hash_state.update(value_string.size()); + hash_state.update(value_string); } ASTPtr ConstantNode::toASTImpl() const diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index 4ebc53d86af..5730dba9218 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -35,6 +35,12 @@ public: return constant_value->getValue(); } + /// Get constant value string representation + const String & getValueStringRepresentation() const + { + return value_string; + } + ConstantValuePtr getConstantValueOrNull() const override { return constant_value; @@ -47,7 +53,7 @@ public: String getName() const override { - return value_string_dump; + return value_string; } DataTypePtr getResultType() const override @@ -68,7 +74,7 @@ protected: private: ConstantValuePtr constant_value; - String value_string_dump; + String value_string; }; } diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index 519e4a15a7f..7f97aa89178 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -143,16 +143,16 @@ public: return result_type != nullptr && (function != nullptr || aggregate_function != nullptr); } - /// Is window function + /// Is function node window function bool isWindowFunction() const { - return isAggregateFunction() && getWindowNode() != nullptr; + return getWindowNode() != nullptr; } - /// Is function node resolved as aggregate function + /// Is function node aggregate function bool isAggregateFunction() const { - return aggregate_function != nullptr; + return aggregate_function != nullptr && !isWindowFunction(); } /// Is function node ordinary function diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 885ca88ba4f..9d0fd9bd6ac 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -187,10 +187,8 @@ namespace ErrorCodes * Identifier first part value bound to test_table column value, but nested identifier part cannot be resolved from it, * lookup should not be continued, and exception must be thrown because if lookup continues identifier can be resolved from parent scope. * - * TODO: Add expression name into query tree node. Example: SELECT plus(1, 1). Result: SELECT 2. Expression name of constant node should be 2. * TODO: Update exception messages * TODO: JOIN TREE subquery constant columns - * TODO: Column name qualifications * TODO: Table identifiers with optional UUID. * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. @@ -207,7 +205,7 @@ enum class IdentifierLookupContext : uint8_t { EXPRESSION = 0, FUNCTION, - TABLE, + TABLE_EXPRESSION, }; const char * toString(IdentifierLookupContext identifier_lookup_context) @@ -216,7 +214,7 @@ const char * toString(IdentifierLookupContext identifier_lookup_context) { case IdentifierLookupContext::EXPRESSION: return "EXPRESSION"; case IdentifierLookupContext::FUNCTION: return "FUNCTION"; - case IdentifierLookupContext::TABLE: return "TABLE"; + case IdentifierLookupContext::TABLE_EXPRESSION: return "TABLE_EXPRESSION"; } } @@ -226,7 +224,7 @@ const char * toStringLowercase(IdentifierLookupContext identifier_lookup_context { case IdentifierLookupContext::EXPRESSION: return "expression"; case IdentifierLookupContext::FUNCTION: return "function"; - case IdentifierLookupContext::TABLE: return "table"; + case IdentifierLookupContext::TABLE_EXPRESSION: return "table expression"; } } @@ -248,9 +246,9 @@ struct IdentifierLookup return lookup_context == IdentifierLookupContext::FUNCTION; } - bool isTableLookup() const + bool isTableExpressionLookup() const { - return lookup_context == IdentifierLookupContext::TABLE; + return lookup_context == IdentifierLookupContext::TABLE_EXPRESSION; } String dump() const @@ -501,15 +499,132 @@ private: std::unordered_map alias_name_to_expressions_size; }; +/** Projection names is name of query tree node that is used in projection part of query node. + * Example: SELECT id FROM test_table; + * `id` is projection name of column node + * + * Example: SELECT id AS id_alias FROM test_table; + * `id_alias` is projection name of column node + * + * Calculation of projection names is done during expression nodes resolution. This is done this way + * because after identifier node is resolved we lose information about identifier name. We could + * potentially save this information in query tree node itself, but that would require to clone it in some cases. + * Example: SELECT big_scalar_subquery AS a, a AS b, b AS c; + * All 3 nodes in projection are the same big_scalar_subquery, but they have different projection names. + * If we want to save it in query tree node, we have to clone subquery node that could lead to performance degradation. + * + * Possible solution is to separate query node metadata and query node content. So only node metadata could be cloned + * if we want to change projection name. This solution does not seem to be easy for client of query tree because projection + * name will be part of interface. If we potentially could hide projection names calculation in analyzer without introducing additional + * changes in query tree structure that would be preferable. + * + * Currently each resolve method returns projection names array. Resolve method must compute projection names of node. + * If node is resolved as list node this is case for `untuple` function or `matcher` result projection names array must contain projection names + * for result nodes. + * If node is not resolved as list node, projection names array contain single projection name for node. + * + * Rules for projection names: + * 1. If node has alias. It is node projection name. + * Except scenario where `untuple` function has alias. Example: SELECT untuple(expr) AS alias, alias. + * + * 2. For constant it is constant value string representation. + * + * 3. For identifier: + * If identifier is resolved from JOIN TREE, we want to remove additional identifier qualifications. + * Example: SELECT default.test_table.id FROM test_table. + * Result projection name is `id`. + * + * Example: SELECT t1.id FROM test_table_1 AS t1, test_table_2 AS t2 + * In example both test_table_1, test_table_2 have `id` column. + * In such case projection name is `t1.id` because if additional qualification is removed then column projection name `id` will be ambiguous. + * + * Example: SELECT default.test_table_1.id FROM test_table_1 AS t1, test_table_2 AS t2 + * In such case projection name is `test_table_1.id` because we remove unnecessary database qualification, but table name qualification cannot be removed + * because otherwise column projection name `id` will be ambiguous. + * + * If identifier is not resolved from JOIN TREE. Identifier name is projection name. + * Except scenario where `untuple` function resolved using identifier. Example: SELECT untuple(expr) AS alias, alias. + * Example: SELECT sum(1, 1) AS value, value. + * In such case both nodes have `value` projection names. + * + * Example: SELECT id AS value, value FROM test_table. + * In such case both nodes have have `value` projection names. + * + * Special case is `untuple` function. If `untuple` function specified with alias, then result nodes will have alias.tuple_column_name projection names. + * Example: SELECT cast(tuple(1), 'Tuple(id UInt64)') AS value, untuple(value) AS a; + * Result projection names are `value`, `a.id`. + * + * If `untuple` function does not have alias then result nodes will have `tupleElement(untuple_expression_projection_name, 'tuple_column_name') projection names. + * + * Example: SELECT cast(tuple(1), 'Tuple(id UInt64)') AS value, untuple(value); + * Result projection names are `value`, `tupleElement(value, 'id')`; + * + * 4. For function: + * Projection name consists from function_name(parameters_projection_names)(arguments_projection_names). + * Additionally if function is window function. Window node projection name is used with OVER clause. + * Example: function_name (parameters_names)(argument_projection_names) OVER window_name; + * Example: function_name (parameters_names)(argument_projection_names) OVER (PARTITION BY id ORDER BY id). + * Example: function_name (parameters_names)(argument_projection_names) OVER (window_name ORDER BY id). + * + * 5. For lambda: + * If it is standalone lambda that returns single expression, function projection name is used. + * Example: WITH (x -> x + 1) AS lambda SELECT lambda(1). + * Projection name is `lambda(1)`. + * + * If is it standalone lambda that returns list, projection names of list nodes are used. + * Example: WITH (x -> *) AS lambda SELECT lambda(1) FROM test_table; + * If test_table has two columns `id`, `value`. Then result projection names are `id`, `value`. + * + * If lambda is argument of function. + * Then projection name consists from lambda(tuple(lambda_arguments)(lambda_body_projection_name)); + * + * 6. For matcher: + * Matched nodes projection names are used as matcher projection names. + * + * Matched nodes must be qualified if needed. + * Example: SELECT * FROM test_table_1 AS t1, test_table_2 AS t2. + * In example table test_table_1 and test_table_2 both have `id`, `value` columns. + * Matched nodes after unqualified matcher resolve must be qualified to avoid ambiguous projection names. + * Result projection names must be `t1.id`, `t1.value`, `t2.id`, `t2.value`. + * + * There are special cases + * 1. For lambda inside APPLY matcher transformer: + * Example: SELECT * APPLY x -> toString(x) FROM test_table. + * In such case lambda argument projection name `x` will be replaced by matched node projection name. + * If table has two columns `id` and `value`. Then result projection names are `toString(id)`, `toString(value)`; + * + * 2. For unqualified matcher when JOIN tree contains JOIN with USING. + * Example: SELECT * FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 USING(id); + * Result projection names must be `id`, `t1.value`, `t2.value`. + * + * 7. For subquery: + * For subquery projection name consists of `_subquery_` prefix and implementation specific unique number suffix. + * Example: SELECT (SELECT 1), (SELECT 1 UNION DISTINCT SELECT 1); + * Result projection name can be `_subquery_1`, `subquery_2`; + * + * 8. For table: + * Table node can be used in expression context only as right argument of IN function. In that case identifier is used + * as table node projection name. + * Example: SELECT id IN test_table FROM test_table; + * Result projection name is `in(id, test_table)`. + */ +using ProjectionName = String; +using ProjectionNames = std::vector; +constexpr auto PROJECTION_NAME_PLACEHOLDER = "__projection_name_placeholder"; + struct IdentifierResolveScope { /// Construct identifier resolve scope using scope node, and parent scope IdentifierResolveScope(QueryTreeNodePtr scope_node_, IdentifierResolveScope * parent_scope_) : scope_node(std::move(scope_node_)) , parent_scope(parent_scope_) - {} + { + if (parent_scope) + subquery_depth = parent_scope->subquery_depth; + } QueryTreeNodePtr scope_node; + IdentifierResolveScope * parent_scope = nullptr; /// Identifier lookup to result @@ -548,21 +663,12 @@ struct IdentifierResolveScope /// Current scope expression std::unordered_set non_cached_identifier_lookups_during_expression_resolve; - /// Allow to check parent scopes if identifier cannot be resolved in current scope - bool allow_to_check_parent_scopes = true; + /// Table expression node to data + std::unordered_map table_expression_node_to_data; /// Use identifier lookup to result cache bool use_identifier_lookup_to_result_cache = true; - /// Table expression node to data cache - std::unordered_map table_expression_node_to_data; - - /// Stage when names for projection are calculated - bool projection_names_calculation_stage = false; - - /// Node to projection name - std::unordered_map node_to_projection_name; - /// Subquery depth size_t subquery_depth = 0; @@ -645,6 +751,10 @@ struct IdentifierResolveScope for (const auto & [cte_name, node] : cte_name_to_query_node) buffer << "CTE name " << cte_name << " node " << node->formatASTForErrorMessage() << '\n'; + buffer << "WINDOW name to window node table size " << window_name_to_window_node.size() << '\n'; + for (const auto & [window_name, node] : window_name_to_window_node) + buffer << "CTE name " << window_name << " node " << node->formatASTForErrorMessage() << '\n'; + buffer << "Nodes with duplicated aliases size " << nodes_with_duplicated_aliases.size() << '\n'; for (const auto & node : nodes_with_duplicated_aliases) buffer << "Alias name " << node->getAlias() << " node " << node->formatASTForErrorMessage() << '\n'; @@ -652,8 +762,20 @@ struct IdentifierResolveScope buffer << "Expression resolve process stack " << '\n'; expressions_in_resolve_process_stack.dump(buffer); - buffer << "Allow to check parent scopes " << allow_to_check_parent_scopes << '\n'; - // buffer << "Parent scope " << parent_scope << '\n'; + buffer << "Table expressions in resolve process size " << table_expressions_in_resolve_process.size() << '\n'; + for (const auto & node : table_expressions_in_resolve_process) + buffer << "Table expression " << node->formatASTForErrorMessage() << '\n'; + + buffer << "Non cached identifier lookups during expression resolve " << non_cached_identifier_lookups_during_expression_resolve.size() << '\n'; + for (const auto & identifier_lookup : non_cached_identifier_lookups_during_expression_resolve) + buffer << "Identifier lookup " << identifier_lookup.dump() << '\n'; + + buffer << "Table expression node to data " << table_expression_node_to_data.size() << '\n'; + for (const auto & [table_expression_node, table_expression_data] : table_expression_node_to_data) + buffer << "Table expression node " << table_expression_node->formatASTForErrorMessage() << " data " << table_expression_data.dump() << '\n'; + + buffer << "Use identifier lookup to result cache " << use_identifier_lookup_to_result_cache << '\n'; + buffer << "Subquery depth " << subquery_depth << '\n'; } [[maybe_unused]] String dump() const @@ -879,10 +1001,28 @@ private: QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); - void evaluateScalarSubquery(QueryTreeNodePtr & query_tree_node, size_t subquery_depth); + void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, size_t subquery_depth); void validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + static ProjectionName calculateFunctionProjectionName(const QueryTreeNodePtr & function_node, + const ProjectionNames & parameters_projection_names, + const ProjectionNames & arguments_projection_names); + + static ProjectionName calculateWindowProjectionName(const QueryTreeNodePtr & window_node, + const QueryTreeNodePtr & parent_window_node, + const String & parent_window_name, + const ProjectionNames & partition_by_projection_names, + const ProjectionNames & order_by_projection_names, + const ProjectionName & frame_begin_offset_projection_name, + const ProjectionName & frame_end_offset_projection_name); + + static ProjectionName calculateSortColumnProjectionName(const QueryTreeNodePtr & sort_column_node, + const ProjectionName & sort_expression_projection_name, + const ProjectionName & fill_from_expression_projection_name, + const ProjectionName & fill_to_expression_projection_name, + const ProjectionName & fill_step_expression_projection_name); + static void validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope); static void validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); @@ -897,6 +1037,8 @@ private: QueryTreeNodePtr tryResolveIdentifierFromExpressionArguments(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); + static bool tryBindIdentifierToAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); + QueryTreeNodePtr tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, IdentifierResolveSettings identifier_resolve_settings = {}); QueryTreeNodePtr tryResolveIdentifierFromTableColumns(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); @@ -921,32 +1063,32 @@ private: using QueryTreeNodesWithNames = std::vector>; - static void matcherQualifyColumnsForProjectionNamesIfNeeded(QueryTreeNodesWithNames & matched_nodes_with_column_names, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + void qualifyMatchedColumnsProjectionNamesIfNeeded(QueryTreeNodesWithNames & matched_nodes_with_column_names, + const QueryTreeNodePtr & table_expression_node, + IdentifierResolveScope & scope); QueryTreeNodesWithNames resolveQualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); QueryTreeNodesWithNames resolveUnqualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); - QueryTreeNodePtr resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); + ProjectionNames resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); - void resolveWindow(QueryTreeNodePtr & window_node, IdentifierResolveScope & scope); + ProjectionName resolveWindow(QueryTreeNodePtr & window_node, IdentifierResolveScope & scope); - void resolveLambda(const QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope); + ProjectionNames resolveLambda(const QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope); - void resolveFunction(QueryTreeNodePtr & function_node, IdentifierResolveScope & scope); + ProjectionNames resolveFunction(QueryTreeNodePtr & function_node, IdentifierResolveScope & scope); - void resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); + ProjectionNames resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); - void resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); + ProjectionNames resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); - void resolveSortColumnsNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope); + ProjectionNames resolveSortNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope); void resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpolate_node_list, IdentifierResolveScope & scope); void resolveWindowNodeList(QueryTreeNodePtr & window_node_list, IdentifierResolveScope & scope); - String calculateProjectionNodeDisplayName(QueryTreeNodePtr & node, IdentifierResolveScope & scope); - NamesAndTypes resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope); void initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope); @@ -967,6 +1109,16 @@ private: /// Array join expressions counter size_t array_join_expressions_counter = 0; + + /// Subquery counter + size_t subquery_counter = 0; + + /// Global expression node to projection name map + std::unordered_map node_to_projection_name; + + /// Global resolve expression node to projection names map + std::unordered_map resolved_expressions; + }; /// Utility functions implementation @@ -1011,7 +1163,7 @@ QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunction(const std } /// Evaluate scalar subquery and perform constant folding. -void QueryAnalyzer::evaluateScalarSubquery(QueryTreeNodePtr & node, size_t subquery_depth) +void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size_t subquery_depth) { auto * query_node = node->as(); auto * union_node = node->as(); @@ -1144,6 +1296,182 @@ void QueryAnalyzer::validateJoinTableExpressionWithoutAlias(const QueryTreeNodeP scope.scope_node->formatASTForErrorMessage()); } +ProjectionName QueryAnalyzer::calculateFunctionProjectionName(const QueryTreeNodePtr & function_node, const ProjectionNames & parameters_projection_names, + const ProjectionNames & arguments_projection_names) +{ + const auto & function_node_typed = function_node->as(); + + WriteBufferFromOwnString buffer; + buffer << function_node_typed.getFunctionName(); + + if (!parameters_projection_names.empty()) + { + buffer << '('; + + size_t function_parameters_projection_names_size = parameters_projection_names.size(); + for (size_t i = 0; i < function_parameters_projection_names_size; ++i) + { + buffer << parameters_projection_names[i]; + + if (i + 1 != function_parameters_projection_names_size) + buffer << ", "; + } + + buffer << ')'; + } + + buffer << '('; + + size_t function_arguments_projection_names_size = arguments_projection_names.size(); + for (size_t i = 0; i < function_arguments_projection_names_size; ++i) + { + buffer << arguments_projection_names[i]; + + if (i + 1 != function_arguments_projection_names_size) + buffer << ", "; + } + + buffer << ')'; + + return buffer.str(); +} + +ProjectionName QueryAnalyzer::calculateWindowProjectionName(const QueryTreeNodePtr & window_node, + const QueryTreeNodePtr & parent_window_node, + const String & parent_window_name, + const ProjectionNames & partition_by_projection_names, + const ProjectionNames & order_by_projection_names, + const ProjectionName & frame_begin_offset_projection_name, + const ProjectionName & frame_end_offset_projection_name) +{ + const auto & window_node_typed = window_node->as(); + const auto & window_frame = window_node_typed.getWindowFrame(); + + bool parent_window_node_has_partition_by = false; + bool parent_window_node_has_order_by = false; + + if (parent_window_node) + { + const auto & parent_window_node_typed = parent_window_node->as(); + parent_window_node_has_partition_by = parent_window_node_typed.hasPartitionBy(); + parent_window_node_has_order_by = parent_window_node_typed.hasOrderBy(); + } + + WriteBufferFromOwnString buffer; + + if (!parent_window_name.empty()) + buffer << parent_window_name; + + if (!partition_by_projection_names.empty() && !parent_window_node_has_partition_by) + { + if (!parent_window_name.empty()) + buffer << ' '; + + buffer << "PARTITION BY "; + + size_t partition_by_projection_names_size = partition_by_projection_names.size(); + for (size_t i = 0; i < partition_by_projection_names_size; ++i) + { + buffer << partition_by_projection_names[i]; + if (i + 1 != partition_by_projection_names_size) + buffer << ", "; + } + } + + if (!order_by_projection_names.empty() && !parent_window_node_has_order_by) + { + if (!partition_by_projection_names.empty() || !parent_window_name.empty()) + buffer << ' '; + + buffer << "ORDER BY "; + + size_t order_by_projection_names_size = order_by_projection_names.size(); + for (size_t i = 0; i < order_by_projection_names_size; ++i) + { + buffer << order_by_projection_names[i]; + if (i + 1 != order_by_projection_names_size) + buffer << ", "; + } + } + + if (!window_frame.is_default) + { + if (!partition_by_projection_names.empty() || !order_by_projection_names.empty() || !parent_window_name.empty()) + buffer << ' '; + + buffer << window_frame.type << " BETWEEN "; + if (window_frame.begin_type == WindowFrame::BoundaryType::Current) + { + buffer << "CURRENT ROW"; + } + else if (window_frame.begin_type == WindowFrame::BoundaryType::Unbounded) + { + buffer << "UNBOUNDED"; + buffer << " " << (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); + } + else + { + buffer << frame_begin_offset_projection_name; + buffer << " " << (window_frame.begin_preceding ? "PRECEDING" : "FOLLOWING"); + } + + buffer << " AND "; + + if (window_frame.end_type == WindowFrame::BoundaryType::Current) + { + buffer << "CURRENT ROW"; + } + else if (window_frame.end_type == WindowFrame::BoundaryType::Unbounded) + { + buffer << "UNBOUNDED"; + buffer << " " << (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); + } + else + { + buffer << frame_end_offset_projection_name; + buffer << " " << (window_frame.end_preceding ? "PRECEDING" : "FOLLOWING"); + } + } + + return buffer.str(); +} + +ProjectionName QueryAnalyzer::calculateSortColumnProjectionName(const QueryTreeNodePtr & sort_column_node, const ProjectionName & sort_expression_projection_name, + const ProjectionName & fill_from_expression_projection_name, const ProjectionName & fill_to_expression_projection_name, const ProjectionName & fill_step_expression_projection_name) +{ + auto & sort_node_typed = sort_column_node->as(); + + WriteBufferFromOwnString sort_column_projection_name_buffer; + sort_column_projection_name_buffer << sort_expression_projection_name; + + auto sort_direction = sort_node_typed.getSortDirection(); + sort_column_projection_name_buffer << (sort_direction == SortDirection::ASCENDING ? " ASC" : " DESC"); + + auto nulls_sort_direction = sort_node_typed.getNullsSortDirection(); + + if (nulls_sort_direction) + sort_column_projection_name_buffer << " NULLS " << (nulls_sort_direction == sort_direction ? "LAST" : "FIRST"); + + if (auto collator = sort_node_typed.getCollator()) + sort_column_projection_name_buffer << " COLLATE " << collator->getLocale(); + + if (sort_node_typed.withFill()) + { + sort_column_projection_name_buffer << " WITH FILL"; + + if (sort_node_typed.hasFillFrom()) + sort_column_projection_name_buffer << " FROM " << fill_from_expression_projection_name; + + if (sort_node_typed.hasFillTo()) + sort_column_projection_name_buffer << " TO " << fill_to_expression_projection_name; + + if (sort_node_typed.hasFillStep()) + sort_column_projection_name_buffer << " STEP " << fill_step_expression_projection_name; + } + + return sort_column_projection_name_buffer.str(); +} + void QueryAnalyzer::validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope) { const auto limit_offset_constant_value = expression_node->getConstantValueOrNull(); @@ -1228,14 +1556,15 @@ void QueryAnalyzer::mergeWindowWithParentWindow(const QueryTreeNodePtr & window_ auto & parent_window_node_typed = parent_window_node->as(); - // If an existing_window_name is specified it must refer to an earlier - // entry in the WINDOW list; the new window copies its partitioning clause - // from that entry, as well as its ordering clause if any. In this case - // the new window cannot specify its own PARTITION BY clause, and it can - // specify ORDER BY only if the copied window does not have one. The new - // window always uses its own frame clause; the copied window must not - // specify a frame clause. - // -- https://www.postgresql.org/docs/current/sql-select.html + /** If an existing_window_name is specified it must refer to an earlier + * entry in the WINDOW list; the new window copies its partitioning clause + * from that entry, as well as its ordering clause if any. In this case + * the new window cannot specify its own PARTITION BY clause, and it can + * specify ORDER BY only if the copied window does not have one. The new + * window always uses its own frame clause; the copied window must not + * specify a frame clause. + * https://www.postgresql.org/docs/current/sql-select.html + */ if (window_node_typed.hasPartitionBy()) { throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -1373,7 +1702,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(cons if (identifier_lookup.isExpressionLookup() && node_type != QueryTreeNodeType::COLUMN && node_type != QueryTreeNodeType::CONSTANT && node_type != QueryTreeNodeType::FUNCTION && node_type != QueryTreeNodeType::QUERY && node_type != QueryTreeNodeType::UNION) return {}; - else if (identifier_lookup.isTableLookup() && node_type != QueryTreeNodeType::TABLE && node_type != QueryTreeNodeType::TABLE_FUNCTION && + else if (identifier_lookup.isTableExpressionLookup() && node_type != QueryTreeNodeType::TABLE && node_type != QueryTreeNodeType::TABLE_FUNCTION && node_type != QueryTreeNodeType::QUERY && node_type != QueryTreeNodeType::UNION) return {}; else if (identifier_lookup.isFunctionLookup() && node_type != QueryTreeNodeType::LAMBDA) @@ -1393,6 +1722,29 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(cons return it->second; } +bool QueryAnalyzer::tryBindIdentifierToAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +{ + const auto & identifier_bind_part = identifier_lookup.identifier.front(); + + auto get_alias_name_to_node_map = [&]() -> std::unordered_map & + { + if (identifier_lookup.isExpressionLookup()) + return scope.alias_name_to_expression_node; + else if (identifier_lookup.isFunctionLookup()) + return scope.alias_name_to_lambda_node; + + return scope.alias_name_to_table_expression_node; + }; + + auto & alias_name_to_node_map = get_alias_name_to_node_map(); + auto it = alias_name_to_node_map.find(identifier_bind_part); + + if (it == alias_name_to_node_map.end()) + return false; + + return true; +} + /** Resolve identifier from scope aliases. * * Resolve strategy: @@ -1481,13 +1833,13 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier return {}; } - scope.expressions_in_resolve_process_stack.pushNode(it->second); - auto node_type = it->second->getNodeType(); /// Resolve expression if necessary if (node_type == QueryTreeNodeType::IDENTIFIER) { + scope.expressions_in_resolve_process_stack.pushNode(it->second); + auto & alias_identifier_node = it->second->as(); auto identifier = alias_identifier_node.getIdentifier(); auto lookup_result = tryResolveIdentifier(IdentifierLookup{identifier, identifier_lookup.lookup_context}, scope, identifier_resolve_settings); @@ -1505,27 +1857,18 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier scope.alias_name_to_lambda_node.erase(identifier_bind_part); else if (identifier_lookup.isFunctionLookup() && it->second) scope.alias_name_to_expression_node.erase(identifier_bind_part); + + scope.expressions_in_resolve_process_stack.popNode(); } else if (node_type == QueryTreeNodeType::FUNCTION) { - resolveFunction(it->second, scope); + resolveExpressionNode(it->second, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } else if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) { - IdentifierResolveScope subquery_scope(it->second, &scope /*parent_scope*/); - subquery_scope.subquery_depth = scope.subquery_depth + 1; - - if (node_type == QueryTreeNodeType::QUERY) - resolveQuery(it->second, subquery_scope); - else if (node_type == QueryTreeNodeType::UNION) - resolveUnion(it->second, subquery_scope); - - if (identifier_lookup.isExpressionLookup()) - evaluateScalarSubquery(it->second, subquery_scope.subquery_depth); + resolveExpressionNode(it->second, scope, false /*allow_lambda_expression*/, identifier_lookup.isTableExpressionLookup() /*allow_table_expression*/); } - scope.expressions_in_resolve_process_stack.popNode(); - QueryTreeNodePtr result = it->second; if (identifier_lookup.identifier.isCompound() && identifier_lookup.isExpressionLookup() && result) @@ -1610,7 +1953,7 @@ bool QueryAnalyzer::tryBindIdentifierToTableExpression(const IdentifierLookup & const auto & table_name = table_expression_data.table_name; const auto & database_name = table_expression_data.database_name; - if (identifier_lookup.isTableLookup()) + if (identifier_lookup.isTableExpressionLookup()) { size_t parts_size = identifier_lookup.identifier.getPartsSize(); if (parts_size != 1 && parts_size != 2) @@ -1663,7 +2006,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id auto & table_expression_data = scope.getTableExpressionDataOrThrow(table_expression_node); - if (identifier_lookup.isTableLookup()) + if (identifier_lookup.isTableExpressionLookup()) { size_t parts_size = identifier_lookup.identifier.getPartsSize(); if (parts_size != 1 && parts_size != 2) @@ -1724,7 +2067,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id scope.scope_node->formatASTForErrorMessage()); QueryTreeNodePtr result_expression = result_column; - bool projection_name_clone_is_needed = true; + bool clone_is_needed = true; if (!match_full_identifier && compound_identifier) { @@ -1733,55 +2076,52 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id auto tuple_element_result = wrapExpressionNodeInTupleElement(result_expression, identifier_view); resolveFunction(tuple_element_result, scope); result_expression = std::move(tuple_element_result); - projection_name_clone_is_needed = false; + clone_is_needed = false; } - if (scope.projection_names_calculation_stage && scope.table_expression_node_to_data.size() > 1) + if (clone_is_needed) + result_expression = result_expression->clone(); + + auto qualified_identifier = identifier; + for (size_t i = 0; i < identifier_column_qualifier_parts; ++i) { - if (projection_name_clone_is_needed) - result_expression = result_expression->clone(); + auto qualified_identifier_with_removed_part = qualified_identifier; + qualified_identifier_with_removed_part.popFirst(); - auto qualified_identifier = identifier; - for (size_t i = 0; i < identifier_column_qualifier_parts; ++i) + if (qualified_identifier_with_removed_part.empty()) + break; + + if (context->getSettingsRef().prefer_column_name_to_alias + && scope.alias_name_to_expression_node.contains(qualified_identifier_with_removed_part[0])) + break; + + bool can_remove_qualificator = true; + + for (auto & table_expression_to_check_data : scope.table_expression_node_to_data) { - auto qualified_identifier_with_removed_part = qualified_identifier; - qualified_identifier_with_removed_part.popFirst(); + const auto & table_expression_to_check = table_expression_to_check_data.first; + if (table_expression_to_check.get() == table_expression_node.get()) + continue; - if (qualified_identifier_with_removed_part.empty()) - break; + IdentifierLookup column_identifier_lookup{qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION}; + bool can_bind_identifier_to_table_expression = tryBindIdentifierToTableExpression(column_identifier_lookup, table_expression_to_check, scope); - if (context->getSettingsRef().prefer_column_name_to_alias - && scope.alias_name_to_expression_node.contains(qualified_identifier_with_removed_part[0])) - break; - - bool can_remove_qualificator = true; - - for (auto & table_expression_to_check_data : scope.table_expression_node_to_data) + if (can_bind_identifier_to_table_expression) { - const auto & table_expression_to_check = table_expression_to_check_data.first; - if (table_expression_to_check.get() == table_expression_node.get()) - continue; - - IdentifierLookup column_identifier_lookup {qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION}; - bool can_bind_identifier_to_table_expression = tryBindIdentifierToTableExpression(column_identifier_lookup, table_expression_to_check, scope); - - if (can_bind_identifier_to_table_expression) - { - can_remove_qualificator = false; - break; - } - } - - if (!can_remove_qualificator) + can_remove_qualificator = false; break; - - qualified_identifier = std::move(qualified_identifier_with_removed_part); + } } - auto qualified_identifier_full_name = qualified_identifier.getFullName(); - scope.node_to_projection_name.emplace(result_expression, std::move(qualified_identifier_full_name)); + if (!can_remove_qualificator) + break; + + qualified_identifier = std::move(qualified_identifier_with_removed_part); } + auto qualified_identifier_full_name = qualified_identifier.getFullName(); + node_to_projection_name.emplace(result_expression, std::move(qualified_identifier_full_name)); + return result_expression; }; @@ -2058,9 +2398,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTree(const Identifie */ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) { - if (!scope.allow_to_check_parent_scopes) - return {}; - bool initial_scope_is_query = scope.scope_node->getNodeType() == QueryTreeNodeType::QUERY; bool initial_scope_is_expression = !initial_scope_is_query; @@ -2103,7 +2440,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const * During child scope table identifier resolve a, table node test_table with alias a from parent scope * is invalid. */ - if (identifier_lookup.isTableLookup() && !is_cte) + if (identifier_lookup.isTableExpressionLookup() && !is_cte) continue; if (is_cte) @@ -2227,7 +2564,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook } } - if (!resolve_result.resolved_identifier && identifier_lookup.isTableLookup()) + if (!resolve_result.resolved_identifier && identifier_lookup.isTableExpressionLookup()) { auto cte_query_node_it = scope.cte_name_to_query_node.find(identifier_lookup.identifier.getFullName()); if (cte_query_node_it != scope.cte_name_to_query_node.end()) @@ -2249,7 +2586,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook /// Try to resolve table identifier from database catalog - if (!resolve_result.resolved_identifier && identifier_resolve_settings.allow_to_check_database_catalog && identifier_lookup.isTableLookup()) + if (!resolve_result.resolved_identifier && identifier_resolve_settings.allow_to_check_database_catalog && identifier_lookup.isTableExpressionLookup()) { resolve_result.resolved_identifier = tryResolveTableIdentifierFromDatabaseCatalog(identifier_lookup.identifier); @@ -2269,37 +2606,44 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook /// Resolve query tree nodes functions implementation - -void QueryAnalyzer::matcherQualifyColumnsForProjectionNamesIfNeeded(QueryTreeNodesWithNames & matched_nodes_with_column_names, +/** Qualify matched columns projection names for unqualified matcher or qualified matcher resolved nodes + * + * Example: SELECT * FROM test_table AS t1, test_table AS t2; + */ +void QueryAnalyzer::qualifyMatchedColumnsProjectionNamesIfNeeded(QueryTreeNodesWithNames & matched_nodes_with_column_names, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) { - if (!scope.projection_names_calculation_stage || scope.table_expression_node_to_data.size() < 2) - return; - - std::vector qualify_identifier_parts; + /// Build additional column qualification parts array + std::vector additional_column_qualification_parts; if (table_expression_node->hasAlias()) - qualify_identifier_parts = {table_expression_node->getAlias()}; + additional_column_qualification_parts = {table_expression_node->getAlias()}; else if (auto * table_node = table_expression_node->as()) - qualify_identifier_parts = {table_node->getStorageID().getDatabaseName(), table_node->getStorageID().getTableName()}; + additional_column_qualification_parts = {table_node->getStorageID().getDatabaseName(), table_node->getStorageID().getTableName()}; - size_t qualify_identifier_parts_size = qualify_identifier_parts.size(); + size_t additional_column_qualification_parts_size = additional_column_qualification_parts.size(); + + /** For each matched column node iterate over additional column qualifications and apply them if column needs to be qualified. + * To check if column needs to be qualified we check if column name can bind to any other table expression in scope or to scope aliases. + */ + std::vector column_qualified_identifier_parts; for (auto & [column_node, column_name] : matched_nodes_with_column_names) { - std::vector column_qualified_identifier_parts = Identifier(column_name).getParts(); + column_qualified_identifier_parts = Identifier(column_name).getParts(); - for (size_t i = 0; i < qualify_identifier_parts_size; ++i) + /// Iterate over additional column qualifications and apply them if needed + for (size_t i = 0; i < additional_column_qualification_parts_size; ++i) { bool need_to_qualify = false; auto identifier_to_check = Identifier(column_qualified_identifier_parts); + IdentifierLookup lookup{identifier_to_check, IdentifierLookupContext::EXPRESSION}; for (auto & table_expression_data : scope.table_expression_node_to_data) { if (table_expression_data.first.get() == table_expression_node.get()) continue; - IdentifierLookup lookup{identifier_to_check, IdentifierLookupContext::EXPRESSION}; if (tryBindIdentifierToTableExpression(lookup, table_expression_data.first, scope)) { need_to_qualify = true; @@ -2307,10 +2651,18 @@ void QueryAnalyzer::matcherQualifyColumnsForProjectionNamesIfNeeded(QueryTreeNod } } + if (tryBindIdentifierToAliases(lookup, scope)) + need_to_qualify = true; + if (need_to_qualify) { - size_t part_index_to_use_for_qualification = qualify_identifier_parts_size - i - 1; - const auto & part_to_use = qualify_identifier_parts[part_index_to_use_for_qualification]; + /** Add last qualification part that was not used into column qualified identifier. + * If additional column qualification parts consists from [database_name, table_name]. + * On first iteration if column is needed to be qualified to qualify it with table_name. + * On second iteration if column is needed to be qualified to qualify it with database_name. + */ + size_t part_index_to_use_for_qualification = additional_column_qualification_parts_size - i - 1; + const auto & part_to_use = additional_column_qualification_parts[part_index_to_use_for_qualification]; column_qualified_identifier_parts.insert(column_qualified_identifier_parts.begin(), part_to_use); } else @@ -2319,7 +2671,7 @@ void QueryAnalyzer::matcherQualifyColumnsForProjectionNamesIfNeeded(QueryTreeNod } } - scope.node_to_projection_name.emplace(column_node, Identifier(column_qualified_identifier_parts).getFullName()); + node_to_projection_name.emplace(column_node, Identifier(column_qualified_identifier_parts).getFullName()); } } @@ -2360,7 +2712,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu const auto & element_names = tuple_data_type->getElementNames(); - auto matcher_qualified_identifier_copy = matcher_node_typed.getQualifiedIdentifier(); + auto qualified_matcher_element_identifier = matcher_node_typed.getQualifiedIdentifier(); for (const auto & element_name : element_names) { if (!matcher_node_typed.isMatchingColumn(element_name)) @@ -2373,12 +2725,9 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu QueryTreeNodePtr function_query_node = tuple_element_function; resolveFunction(function_query_node, scope); - if (scope.projection_names_calculation_stage) - { - matcher_qualified_identifier_copy.push_back(element_name); - scope.node_to_projection_name.emplace(function_query_node, matcher_qualified_identifier_copy.getFullName()); - matcher_qualified_identifier_copy.pop_back(); - } + qualified_matcher_element_identifier.push_back(element_name); + node_to_projection_name.emplace(function_query_node, qualified_matcher_element_identifier.getFullName()); + qualified_matcher_element_identifier.pop_back(); matched_expression_nodes_with_column_names.emplace_back(std::move(function_query_node), element_name); } @@ -2386,13 +2735,13 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu return matched_expression_nodes_with_column_names; } - /// Try to resolve unqualified matcher for table expression + /// Try to resolve qualified matcher for table expression IdentifierResolveSettings identifier_resolve_settings; identifier_resolve_settings.allow_to_check_cte = false; identifier_resolve_settings.allow_to_check_database_catalog = false; - auto table_identifier_lookup = IdentifierLookup{matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::TABLE}; + auto table_identifier_lookup = IdentifierLookup{matcher_node_typed.getQualifiedIdentifier(), IdentifierLookupContext::TABLE_EXPRESSION}; auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope, identifier_resolve_settings); auto table_expression_node = table_identifier_resolve_result.resolved_identifier; @@ -2438,7 +2787,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu matched_expression_nodes_with_column_names.emplace_back(std::make_shared(column, table_expression_node), column_name); } - matcherQualifyColumnsForProjectionNamesIfNeeded(matched_expression_nodes_with_column_names, table_expression_node, scope); + qualifyMatchedColumnsProjectionNamesIfNeeded(matched_expression_nodes_with_column_names, table_expression_node, scope); return matched_expression_nodes_with_column_names; } @@ -2453,20 +2802,11 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( /** There can be edge case if matcher is inside lambda expression. * Try to find parent query expression using parent scopes. */ - IQueryTreeNode * scope_node = scope.scope_node.get(); - auto * scope_query_node = scope_node->as(); - - while (!scope_query_node) - { - if (!scope.parent_scope) - break; - - scope_node = scope.parent_scope->scope_node.get(); - scope_query_node = scope_node->as(); - } + auto * nearest_query_scope = scope.getNearestQueryScope(); + auto * nearest_query_scope_query_node = nearest_query_scope ? nearest_query_scope->scope_node->as() : nullptr; /// If there are no parent query scope or query scope does not have join tree - if (!scope_query_node || !scope_query_node->getJoinTree()) + if (!nearest_query_scope_query_node || !nearest_query_scope_query_node->getJoinTree()) { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Unqualified matcher {} cannot be resolved. There are no table sources. In scope {}", @@ -2482,7 +2822,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( * expressions that have same names as columns in USING clause must be skipped. */ - auto table_expressions_stack = buildTableExpressionsStack(scope_query_node->getJoinTree()); + auto table_expressions_stack = buildTableExpressionsStack(nearest_query_scope_query_node->getJoinTree()); std::vector table_expressions_column_nodes_with_names_stack; for (auto & table_expression : table_expressions_stack) @@ -2499,9 +2839,9 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( if (join_node) { size_t table_expressions_column_nodes_with_names_stack_size = table_expressions_column_nodes_with_names_stack.size(); - if (table_expressions_column_nodes_with_names_stack_size != 2) + if (table_expressions_column_nodes_with_names_stack_size < 2) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected 2 table expressions on stack before JOIN processing. Actual {}", + "Expected at least 2 table expressions on stack before JOIN processing. Actual {}", table_expressions_column_nodes_with_names_stack_size); auto right_table_expression_columns = std::move(table_expressions_column_nodes_with_names_stack.back()); @@ -2613,7 +2953,15 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), table_expression_column.name); } - matcherQualifyColumnsForProjectionNamesIfNeeded(matched_expression_nodes_with_column_names, table_expression, scope); + qualifyMatchedColumnsProjectionNamesIfNeeded(matched_expression_nodes_with_column_names, table_expression, scope); + + for (auto & [matched_node, column_name] : matched_expression_nodes_with_column_names) + { + auto node_projection_name_it = node_to_projection_name.find(matcher_node); + if (node_projection_name_it != node_to_projection_name.end()) + column_name = node_projection_name_it->second; + } + table_expressions_column_nodes_with_names_stack.push_back(std::move(matched_expression_nodes_with_column_names)); } @@ -2635,7 +2983,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( * 2. Apply column transformers to matched expression nodes. For strict column transformers save used column names. * 3. Validate strict column transformers. */ -QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope) +ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope) { auto & matcher_node_typed = matcher_node->as(); @@ -2660,11 +3008,21 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, }; ListNodePtr list = std::make_shared(); + ProjectionNames result_projection_names; + ProjectionNames node_projection_names; for (auto & [node, column_name] : matched_expression_nodes_with_names) { bool apply_transformer_was_used = false; bool replace_transformer_was_used = false; + bool execute_apply_transformer = false; + bool execute_replace_transformer = false; + + auto projection_name_it = node_to_projection_name.find(node); + if (projection_name_it != node_to_projection_name.end()) + result_projection_names.push_back(projection_name_it->second); + else + result_projection_names.push_back(column_name); for (const auto & transformer : matcher_node_typed.getColumnTransformers().getNodes()) { @@ -2677,24 +3035,8 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, { auto lambda_expression_to_resolve = expression_node->clone(); IdentifierResolveScope lambda_scope(expression_node, &scope /*parent_scope*/); - resolveLambda(lambda_expression_to_resolve, {node}, lambda_scope); + node_projection_names = resolveLambda(lambda_expression_to_resolve, {node}, lambda_scope); auto & lambda_expression_to_resolve_typed = lambda_expression_to_resolve->as(); - - if (auto * lambda_list_node_result = lambda_expression_to_resolve_typed.getExpression()->as()) - { - auto & lambda_list_node_result_nodes = lambda_list_node_result->getNodes(); - size_t lambda_list_node_result_nodes_size = lambda_list_node_result->getNodes().size(); - - if (lambda_list_node_result_nodes_size != 1) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Lambda in APPLY transformer {} resolved as list node with size {}. Expected 1. In scope {}", - apply_transformer->formatASTForErrorMessage(), - lambda_list_node_result_nodes_size, - scope.scope_node->formatASTForErrorMessage()); - - lambda_expression_to_resolve_typed.getExpression() = lambda_list_node_result_nodes[0]; - } - node = lambda_expression_to_resolve_typed.getExpression(); } else if (apply_transformer->getApplyTransformerType() == ApplyColumnTransformerType::FUNCTION) @@ -2702,7 +3044,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, auto function_to_resolve_untyped = expression_node->clone(); auto & function_to_resolve_typed = function_to_resolve_untyped->as(); function_to_resolve_typed.getArguments().getNodes().push_back(node); - resolveFunction(function_to_resolve_untyped, scope); + node_projection_names = resolveFunction(function_to_resolve_untyped, scope); node = function_to_resolve_untyped; } else @@ -2712,6 +3054,8 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, transformer->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } + + execute_apply_transformer = true; } else if (auto * except_transformer = transformer->as()) { @@ -2742,12 +3086,41 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, add_strict_transformer_column_name(replace_transformer, column_name); node = replace_expression->clone(); - resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + node_projection_names = resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + execute_replace_transformer = true; + } + + if (execute_apply_transformer || execute_replace_transformer) + { + if (auto * node_list = node->as()) + { + auto & node_list_nodes = node_list->getNodes(); + size_t node_list_nodes_size = node_list_nodes.size(); + + if (node_list_nodes_size != 1) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "{} transformer {} resolved as list node with size {}. Expected 1. In scope {}", + execute_apply_transformer ? "APPLY" : "REPLACE", + transformer->formatASTForErrorMessage(), + node_list_nodes_size, + scope.scope_node->formatASTForErrorMessage()); + + node = node_list_nodes[0]; + } + + if (node_projection_names.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Matcher node expected 1 projection name. Actual {}", node_projection_names.size()); + + result_projection_names.back() = std::move(node_projection_names[0]); + node_to_projection_name.emplace(node, result_projection_names.back()); + node_projection_names.clear(); } } if (node) list->getNodes().push_back(node); + else + result_projection_names.pop_back(); } for (auto & [strict_transformer, used_column_names] : strict_transformer_to_used_column_names) @@ -2816,7 +3189,9 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, non_matched_column_names_buffer.str()); } - return list; + matcher_node = std::move(list); + + return result_projection_names; } /** Resolve window function window node. @@ -2833,19 +3208,18 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, * If window node has frame begin OFFSET or frame end OFFSET specified, they are resolved, and window node frame constants are updated. * Window node frame is validated. */ -void QueryAnalyzer::resolveWindow(QueryTreeNodePtr & node, IdentifierResolveScope & scope) +ProjectionName QueryAnalyzer::resolveWindow(QueryTreeNodePtr & node, IdentifierResolveScope & scope) { std::string parent_window_name; auto * identifier_node = node->as(); + ProjectionName result_projection_name; + QueryTreeNodePtr parent_window_node; + if (identifier_node) - { parent_window_name = identifier_node->getIdentifier().getFullName(); - } else if (auto * window_node = node->as()) - { parent_window_name = window_node->getParentWindowName(); - } if (!parent_window_name.empty()) { @@ -2863,21 +3237,38 @@ void QueryAnalyzer::resolveWindow(QueryTreeNodePtr & node, IdentifierResolveScop parent_window_name, nearest_query_scope->scope_node->formatASTForErrorMessage()); + parent_window_node = window_node_it->second; + if (identifier_node) - node = window_node_it->second->clone(); + { + node = parent_window_node->clone(); + result_projection_name = parent_window_name; + } else - mergeWindowWithParentWindow(node, window_node_it->second, scope); + { + mergeWindowWithParentWindow(node, parent_window_node, scope); + } } auto & window_node = node->as(); - window_node.setParentWindowName({}); - resolveExpressionNodeList(window_node.getPartitionByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - resolveSortColumnsNodeList(window_node.getOrderByNode(), scope); + + ProjectionNames partition_by_projection_names = resolveExpressionNodeList(window_node.getPartitionByNode(), + scope, + false /*allow_lambda_expression*/, + false /*allow_table_expression*/); + + ProjectionNames order_by_projection_names = resolveSortNodeList(window_node.getOrderByNode(), scope); + + ProjectionNames frame_begin_offset_projection_names; + ProjectionNames frame_end_offset_projection_names; if (window_node.hasFrameBeginOffset()) { - resolveExpressionNode(window_node.getFrameBeginOffsetNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + frame_begin_offset_projection_names = resolveExpressionNode(window_node.getFrameBeginOffsetNode(), + scope, + false /*allow_lambda_expression*/, + false /*allow_table_expression*/); const auto window_frame_begin_constant_value = window_node.getFrameBeginOffsetNode()->getConstantValueOrNull(); if (!window_frame_begin_constant_value || !isNativeNumber(removeNullable(window_frame_begin_constant_value->getType()))) @@ -2887,11 +3278,18 @@ void QueryAnalyzer::resolveWindow(QueryTreeNodePtr & node, IdentifierResolveScop scope.scope_node->formatASTForErrorMessage()); window_node.getWindowFrame().begin_offset = window_frame_begin_constant_value->getValue(); + if (frame_begin_offset_projection_names.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Window FRAME begin offset expected 1 projection name. Actual {}", + frame_begin_offset_projection_names.size()); } if (window_node.hasFrameEndOffset()) { - resolveExpressionNode(window_node.getFrameEndOffsetNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + frame_end_offset_projection_names = resolveExpressionNode(window_node.getFrameEndOffsetNode(), + scope, + false /*allow_lambda_expression*/, + false /*allow_table_expression*/); const auto window_frame_end_constant_value = window_node.getFrameEndOffsetNode()->getConstantValueOrNull(); if (!window_frame_end_constant_value || !isNativeNumber(removeNullable(window_frame_end_constant_value->getType()))) @@ -2901,15 +3299,34 @@ void QueryAnalyzer::resolveWindow(QueryTreeNodePtr & node, IdentifierResolveScop scope.scope_node->formatASTForErrorMessage()); window_node.getWindowFrame().end_offset = window_frame_end_constant_value->getValue(); + if (frame_end_offset_projection_names.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Window FRAME begin offset expected 1 projection name. Actual {}", + frame_end_offset_projection_names.size()); } window_node.getWindowFrame().checkValid(); + + if (result_projection_name.empty()) + { + result_projection_name = calculateWindowProjectionName(node, + parent_window_node, + parent_window_name, + partition_by_projection_names, + order_by_projection_names, + frame_begin_offset_projection_names.empty() ? "" : frame_begin_offset_projection_names.front(), + frame_end_offset_projection_names.empty() ? "" : frame_end_offset_projection_names.front()); + } + + return result_projection_name; } /** Resolve lambda function. * This function modified lambda_node during resolve. It is caller responsibility to clone lambda before resolve * if it is needed for later use. * + * Lambda body expression result projection names is used as lambda projection names. + * * Lambda expression can be resolved into list node. It is caller responsibility to handle it properly. * * lambda_node - node that must have LambdaNode type. @@ -2924,7 +3341,7 @@ void QueryAnalyzer::resolveWindow(QueryTreeNodePtr & node, IdentifierResolveScop * 5. Resolve lambda body expression. * 6. Deregister lambda from lambdas in resolve process. */ -void QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope) +ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope) { auto & lambda = lambda_node->as(); auto & lambda_arguments_nodes = lambda.getArguments().getNodes(); @@ -2986,9 +3403,11 @@ void QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const Qu lambda.getArgumentsNode() = std::move(lambda_new_arguments); /// Lambda body expression is resolved as standard query expression node. - resolveExpressionNode(lambda.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + auto result_projection_names = resolveExpressionNode(lambda.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); lambdas_in_resolve_process.erase(lambda_node.get()); + + return result_projection_names; } /** Resolve function node in scope. @@ -3016,17 +3435,17 @@ void QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const Qu * they must be resolved. * 9. If function is suitable for constant folding, try to perform constant folding for function node. */ -void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveScope & scope) +ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveScope & scope) { FunctionNodePtr function_node_ptr = std::static_pointer_cast(node); - if (function_node_ptr->isResolved()) - return; - auto function_name = function_node_ptr->getFunctionName(); /// Resolve function parameters - resolveExpressionNodeList(function_node_ptr->getParametersNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + auto parameters_projection_names = resolveExpressionNodeList(function_node_ptr->getParametersNode(), + scope, + false /*allow_lambda_expression*/, + false /*allow_table_expression*/); /// Convert function parameters into constant parameters array @@ -3068,8 +3487,12 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc is_special_function_exists = function_name == "exists"; /// Handle SELECT count(*) FROM test_table - if (function_name == "count") - function_node_ptr->getArguments().getNodes().clear(); + if (function_name == "count" && function_node_ptr->getArguments().getNodes().size() == 1) + { + auto * matcher_node = function_node_ptr->getArguments().getNodes().front()->as(); + if (matcher_node && matcher_node->isUnqualified()) + function_node_ptr->getArguments().getNodes().clear(); + } } /** Special functions dictGet and its variations and joinGet can be executed when first argument is identifier. @@ -3099,7 +3522,10 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc /// Resolve function arguments bool allow_table_expressions = is_special_function_in || is_special_function_exists; - resolveExpressionNodeList(function_node_ptr->getArgumentsNode(), scope, true /*allow_lambda_expression*/, allow_table_expressions /*allow_table_expression*/); + auto arguments_projection_names = resolveExpressionNodeList(function_node_ptr->getArgumentsNode(), + scope, + true /*allow_lambda_expression*/, + allow_table_expressions /*allow_table_expression*/); if (is_special_function_exists) { @@ -3205,7 +3631,8 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc argument_column.type = std::make_shared(DataTypes(lambda_arguments_size, nullptr), nullptr); function_lambda_arguments_indexes.push_back(function_argument_index); } - else if (is_special_function_in && (function_argument->getNodeType() == QueryTreeNodeType::TABLE || + else if (is_special_function_in && + (function_argument->getNodeType() == QueryTreeNodeType::TABLE || function_argument->getNodeType() == QueryTreeNodeType::QUERY || function_argument->getNodeType() == QueryTreeNodeType::UNION)) { @@ -3236,6 +3663,9 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc argument_columns.emplace_back(std::move(argument_column)); } + /// Calculate function projection name + ProjectionNames result_projection_names = {calculateFunctionProjectionName(node, parameters_projection_names, arguments_projection_names)}; + /** Try to resolve function as * 1. Lambda function in current scope. Example: WITH (x -> x + 1) AS lambda SELECT lambda(1); * 2. Lambda function from sql user defined functions. @@ -3273,11 +3703,15 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc auto lambda_expression_clone = lambda_expression_untyped->clone(); IdentifierResolveScope lambda_scope(lambda_expression_clone, &scope /*parent_scope*/); - resolveLambda(lambda_expression_clone, function_arguments, lambda_scope); + ProjectionNames lambda_projection_names = resolveLambda(lambda_expression_clone, function_arguments, lambda_scope); auto & resolved_lambda = lambda_expression_clone->as(); node = resolved_lambda.getExpression(); - return; + + if (node->getNodeType() == QueryTreeNodeType::LIST) + result_projection_names = std::move(lambda_projection_names); + + return result_projection_names; } if (function_name == "untuple") @@ -3289,8 +3723,8 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc "Function 'untuple' must have 1 argument. In scope {}", scope.scope_node->formatASTForErrorMessage()); - const auto & tuple_argument = function_arguments[0]; - auto result_type = tuple_argument->getResultType(); + const auto & untuple_argument = function_arguments[0]; + auto result_type = untuple_argument->getResultType(); const auto * tuple_data_type = typeid_cast(result_type.get()); if (!tuple_data_type) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, @@ -3306,20 +3740,28 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc for (const auto & element_name : element_names) { auto tuple_element_function = std::make_shared("tupleElement"); - tuple_element_function->getArguments().getNodes().push_back(tuple_argument); + tuple_element_function->getArguments().getNodes().push_back(untuple_argument); tuple_element_function->getArguments().getNodes().push_back(std::make_shared(element_name)); QueryTreeNodePtr function_query_node = tuple_element_function; resolveFunction(function_query_node, scope); - if (scope.projection_names_calculation_stage && node->hasAlias()) - scope.node_to_projection_name.emplace(function_query_node, node->getAlias() + '.' + element_name); - result_list->getNodes().push_back(std::move(function_query_node)); } - node = result_list; - return; + auto untuple_argument_projection_name = arguments_projection_names.at(0); + result_projection_names.clear(); + + for (const auto & element_name : element_names) + { + if (node->hasAlias()) + result_projection_names.push_back(node->getAlias() + '.' + element_name); + else + result_projection_names.push_back(fmt::format("tupleElement({}, '{}')", untuple_argument_projection_name, element_name)); + } + + node = std::move(result_list); + return result_projection_names; } else if (function_name == "grouping") { @@ -3336,7 +3778,7 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc auto grouping_function = std::make_shared(force_grouping_standard_compatibility); auto grouping_function_adaptor = std::make_shared(std::move(grouping_function)); function_node.resolveAsFunction(std::move(grouping_function_adaptor), std::make_shared()); - return; + return result_projection_names; } } @@ -3352,8 +3794,16 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc auto aggregate_function = AggregateFunctionFactory::instance().get(function_name, argument_types, parameters, properties); function_node.resolveAsWindowFunction(aggregate_function, aggregate_function->getReturnType()); - resolveWindow(function_node.getWindowNode(), scope); - return; + + bool window_node_is_identifier = function_node.getWindowNode()->getNodeType() == QueryTreeNodeType::IDENTIFIER; + ProjectionName window_projection_name = resolveWindow(function_node.getWindowNode(), scope); + + if (window_node_is_identifier) + result_projection_names[0] += " OVER " + window_projection_name; + else + result_projection_names[0] += " OVER (" + window_projection_name + ')'; + + return result_projection_names; } FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, context, parameters); @@ -3364,7 +3814,7 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc if (!function) { if (!AggregateFunctionFactory::instance().isAggregateFunctionName(function_name)) - throw Exception(ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION, + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Function with name {} does not exists. In scope {}", function_name, scope.scope_node->formatASTForErrorMessage()); @@ -3372,7 +3822,7 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc AggregateFunctionProperties properties; auto aggregate_function = AggregateFunctionFactory::instance().get(function_name, argument_types, parameters, properties); function_node.resolveAsAggregateFunction(aggregate_function, aggregate_function->getReturnType()); - return; + return result_projection_names; } /** For lambda arguments we need to initialize lambda argument types DataTypeFunction using `getLambdaArgumentTypes` function. @@ -3386,6 +3836,7 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc { function->getLambdaArgumentTypes(argument_types); + ProjectionNames lambda_projection_names; for (auto & function_lambda_argument_index : function_lambda_arguments_indexes) { auto lambda_to_resolve = function_arguments[function_lambda_argument_index]->clone(); @@ -3425,11 +3876,10 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc } IdentifierResolveScope lambda_scope(lambda_to_resolve, &scope /*parent_scope*/); - resolveLambda(lambda_to_resolve, lambda_arguments, lambda_scope); + lambda_projection_names = resolveLambda(lambda_to_resolve, lambda_arguments, lambda_scope); if (auto * lambda_list_node_result = lambda_to_resolve_typed.getExpression()->as()) { - auto & lambda_list_node_result_nodes = lambda_list_node_result->getNodes(); size_t lambda_list_node_result_nodes_size = lambda_list_node_result->getNodes().size(); if (lambda_list_node_result_nodes_size != 1) @@ -3438,13 +3888,48 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc lambda_list_node_result_nodes_size, lambda_to_resolve->formatASTForErrorMessage()); - lambda_to_resolve_typed.getExpression() = lambda_list_node_result_nodes[0]; + lambda_to_resolve_typed.getExpression() = lambda_list_node_result->getNodes().front(); + } + + if (arguments_projection_names.at(function_lambda_argument_index) == PROJECTION_NAME_PLACEHOLDER) + { + size_t lambda_projection_names_size =lambda_projection_names.size(); + if (lambda_projection_names_size != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Lambda argument inside function expected to have 1 projection name. Actual {}", + lambda_projection_names_size); + + WriteBufferFromOwnString lambda_argument_projection_name_buffer; + lambda_argument_projection_name_buffer << "lambda("; + lambda_argument_projection_name_buffer << "tuple("; + + size_t lambda_argument_names_size = lambda_argument_names.size(); + + for (size_t i = 0; i < lambda_argument_names_size; ++i) + { + const auto & lambda_argument_name = lambda_argument_names[i]; + lambda_argument_projection_name_buffer << lambda_argument_name; + + if (i + 1 != lambda_argument_names_size) + lambda_argument_projection_name_buffer << ", "; + } + + lambda_argument_projection_name_buffer << "), "; + lambda_argument_projection_name_buffer << lambda_projection_names[0]; + lambda_argument_projection_name_buffer << ")"; + + lambda_projection_names.clear(); + + arguments_projection_names[function_lambda_argument_index] = lambda_argument_projection_name_buffer.str(); } argument_types[function_lambda_argument_index] = std::make_shared(function_data_type_argument_types, lambda_to_resolve->getResultType()); argument_columns[function_lambda_argument_index].type = argument_types[function_lambda_argument_index]; function_arguments[function_lambda_argument_index] = std::move(lambda_to_resolve); } + + /// Recalculate function projection name after lambda resolution + result_projection_names = {calculateFunctionProjectionName(node, parameters_projection_names, arguments_projection_names)}; } /** Create SET column for special function IN to allow constant folding @@ -3515,6 +4000,8 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc } function_node.resolveAsFunction(std::move(function), std::move(result_type)); + + return result_projection_names; } /** Resolve expression node. @@ -3543,9 +4030,39 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc * * 4. If node has alias, update its value in scope alias map. Deregister alias from expression_aliases_in_resolve_process. */ -void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) +ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) { + auto resolved_expression_it = resolved_expressions.find(node); + if (resolved_expression_it != resolved_expressions.end()) + { + /** There can be edge case, when subquery for IN function is resolved multiple times in different context. + * SELECT id IN (subquery AS value), value FROM test_table; + * When we start to resolve `value` identifier, subquery is already resolved but constant folding is not performed. + */ + auto node_type = node->getNodeType(); + if (!allow_table_expression && (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION)) + { + IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); + subquery_scope.subquery_depth = scope.subquery_depth + 1; + evaluateScalarSubqueryIfNeeded(node, subquery_scope.subquery_depth); + } + + return resolved_expression_it->second; + } + String node_alias = node->getAlias(); + ProjectionNames result_projection_names; + + if (node_alias.empty()) + { + auto projection_name_it = node_to_projection_name.find(node); + if (projection_name_it != node_to_projection_name.end()) + result_projection_names.push_back(projection_name_it->second); + } + else + { + result_projection_names.push_back(node_alias); + } /** Do not use alias table if node has alias same as some other node. * Example: WITH x -> x + 1 AS lambda SELECT 1 AS lambda; @@ -3592,6 +4109,14 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes auto resolve_identifier_expression_result = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::EXPRESSION}, scope); node = resolve_identifier_expression_result.resolved_identifier; + if (node && result_projection_names.empty() && + (resolve_identifier_expression_result.isResolvedFromJoinTree() || resolve_identifier_expression_result.isResolvedFromExpressionArguments())) + { + auto projection_name_it = node_to_projection_name.find(node); + if (projection_name_it != node_to_projection_name.end()) + result_projection_names.push_back(projection_name_it->second); + } + if (node && !node_alias.empty()) scope.alias_name_to_lambda_node.erase(node_alias); @@ -3605,7 +4130,7 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes if (!node && allow_table_expression) { - node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::TABLE}, scope).resolved_identifier; + node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::TABLE_EXPRESSION}, scope).resolved_identifier; /// If table identifier is resolved as CTE clone it bool resolved_as_cte = node && node->as() && node->as()->isCTE(); @@ -3624,7 +4149,7 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes message_clarification = std::string(" or ") + toStringLowercase(IdentifierLookupContext::FUNCTION); if (allow_table_expression) - message_clarification = std::string(" or ") + toStringLowercase(IdentifierLookupContext::TABLE); + message_clarification = std::string(" or ") + toStringLowercase(IdentifierLookupContext::TABLE_EXPRESSION); throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown {}{} identifier {} in scope {}", @@ -3634,11 +4159,27 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes scope.scope_node->formatASTForErrorMessage()); } + if (node->getNodeType() == QueryTreeNodeType::LIST) + { + result_projection_names.clear(); + resolved_expression_it = resolved_expressions.find(node); + if (resolved_expression_it != resolved_expressions.end()) + return resolved_expression_it->second; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Identifier {} resolve into list node and list node projection names are not initialized. In scope {}", + unresolved_identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + } + + if (result_projection_names.empty()) + result_projection_names.push_back(unresolved_identifier.getFullName()); + break; } case QueryTreeNodeType::MATCHER: { - node = resolveMatcher(node, scope); + result_projection_names = resolveMatcher(node, scope); break; } case QueryTreeNodeType::TRANSFORMER: @@ -3655,24 +4196,38 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes * Example: SELECT a, untuple(CAST(('hello', 1) AS Tuple(name String, count UInt32))) AS a; * During resolveFunction `untuple` function is replaced by list of 2 constants 'hello', 1. */ - resolveExpressionNodeList(node, scope, allow_lambda_expression, allow_lambda_expression); + result_projection_names = resolveExpressionNodeList(node, scope, allow_lambda_expression, allow_lambda_expression); break; } case QueryTreeNodeType::CONSTANT: { + if (result_projection_names.empty()) + { + const auto & constant_node = node->as(); + result_projection_names.push_back(constant_node.getValueStringRepresentation()); + } + /// Already resolved break; } case QueryTreeNodeType::COLUMN: { - auto & column_function_node = node->as(); - if (column_function_node.hasExpression()) - resolveExpressionNode(column_function_node.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + auto & column_node = node->as(); + if (column_node.hasExpression()) + resolveExpressionNode(column_node.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + if (result_projection_names.empty()) + result_projection_names.push_back(column_node.getColumnName()); + break; } case QueryTreeNodeType::FUNCTION: { - resolveFunction(node, scope); + auto function_projection_names = resolveFunction(node, scope); + + if (result_projection_names.empty() || node->getNodeType() == QueryTreeNodeType::LIST) + result_projection_names = std::move(function_projection_names); + break; } case QueryTreeNodeType::LAMBDA: @@ -3683,6 +4238,9 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); + if (result_projection_names.empty()) + result_projection_names.push_back(PROJECTION_NAME_PLACEHOLDER); + /// Lambda must be resolved by caller break; } @@ -3696,7 +4254,7 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes case QueryTreeNodeType::INTERPOLATE: { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Interpolate column {} is not allowed in expression. In scope {}", + "Interpolate {} is not allowed in expression. In scope {}", node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } @@ -3714,36 +4272,37 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes "Table is not allowed {} in expression. In scope {}", node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); + + auto & table_node = node->as(); + result_projection_names.push_back(table_node.getStorageID().getFullNameNotQuoted()); + break; } case QueryTreeNodeType::TABLE_FUNCTION: { - if (!allow_table_expression) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Table function is not allowed {} in expression. In scope {}", - node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - break; + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Table function is not allowed {} in expression. In scope {}", + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); } case QueryTreeNodeType::QUERY: - { - IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); - subquery_scope.subquery_depth = scope.subquery_depth + 1; - resolveQuery(node, subquery_scope); - - if (!allow_table_expression) - evaluateScalarSubquery(node, subquery_scope.subquery_depth); - - break; - } + [[fallthrough]]; case QueryTreeNodeType::UNION: { IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); subquery_scope.subquery_depth = scope.subquery_depth + 1; - resolveUnion(node, subquery_scope); + + if (node_type == QueryTreeNodeType::QUERY) + resolveQuery(node, subquery_scope); + else + resolveUnion(node, subquery_scope); if (!allow_table_expression) - evaluateScalarSubquery(node, subquery_scope.subquery_depth); + evaluateScalarSubqueryIfNeeded(node, subquery_scope.subquery_depth); + + ++subquery_counter; + if (result_projection_names.empty()) + result_projection_names.push_back("_subquery_" + std::to_string(subquery_counter)); break; } @@ -3780,10 +4339,14 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes } } + resolved_expressions.emplace(node, result_projection_names); + scope.expressions_in_resolve_process_stack.popNode(); bool expression_was_root = scope.expressions_in_resolve_process_stack.empty(); if (expression_was_root) scope.non_cached_identifier_lookups_during_expression_resolve.clear(); + + return result_projection_names; } /** Resolve expression node list. @@ -3794,7 +4357,7 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes * Example: CREATE TABLE test_table (id UInt64, value UInt64) ENGINE=TinyLog; SELECT plus(*) FROM test_table; * Example: SELECT *** FROM system.one; */ -void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) +ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) { auto & node_list_typed = node_list->as(); size_t node_list_size = node_list_typed.getNodes().size(); @@ -3802,76 +4365,153 @@ void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, Iden QueryTreeNodes result_nodes; result_nodes.reserve(node_list_size); + ProjectionNames result_projection_names; + for (auto & node : node_list_typed.getNodes()) { auto node_to_resolve = node; - resolveExpressionNode(node_to_resolve, scope, allow_lambda_expression, allow_table_expression); + auto expression_node_projection_names = resolveExpressionNode(node_to_resolve, scope, allow_lambda_expression, allow_table_expression); + size_t expected_projection_names_size = 1; if (auto * expression_list = node_to_resolve->as()) { + expected_projection_names_size = expression_list->getNodes().size(); for (auto & expression_list_node : expression_list->getNodes()) - result_nodes.push_back(std::move(expression_list_node)); + result_nodes.push_back(expression_list_node); } else { result_nodes.push_back(std::move(node_to_resolve)); } + + if (expression_node_projection_names.size() != expected_projection_names_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expression nodes list expected {} projection names. Actual {}", + expected_projection_names_size, + expression_node_projection_names.size()); + + result_projection_names.insert(result_projection_names.end(), expression_node_projection_names.begin(), expression_node_projection_names.end()); + expression_node_projection_names.clear(); } node_list_typed.getNodes() = std::move(result_nodes); + + return result_projection_names; } /** Resolve sort columns nodes list. */ -void QueryAnalyzer::resolveSortColumnsNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope) +ProjectionNames QueryAnalyzer::resolveSortNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope) { - auto & sort_node_list_typed = sort_node_list->as(); + ProjectionNames result_projection_names; + ProjectionNames sort_expression_projection_names; + ProjectionNames fill_from_expression_projection_names; + ProjectionNames fill_to_expression_projection_names; + ProjectionNames fill_step_expression_projection_names; + auto & sort_node_list_typed = sort_node_list->as(); for (auto & node : sort_node_list_typed.getNodes()) { auto & sort_node = node->as(); - resolveExpressionNode(sort_node.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + sort_expression_projection_names = resolveExpressionNode(sort_node.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + if (auto * sort_column_list_node = sort_node.getExpression()->as()) + { + size_t sort_column_list_node_size = sort_column_list_node->getNodes().size(); + if (sort_column_list_node_size != 1) + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Sort column node expression resolved into list with size {}. Expected 1. In scope {}", + sort_column_list_node_size, + scope.scope_node->formatASTForErrorMessage()); + } + + sort_node.getExpression() = sort_column_list_node->getNodes().front(); + } + + size_t sort_expression_projection_names_size = sort_expression_projection_names.size(); + if (sort_expression_projection_names_size != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Sort expression expected 1 projection name. Actual {}", + sort_expression_projection_names_size); if (sort_node.hasFillFrom()) { - resolveExpressionNode(sort_node.getFillFrom(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + fill_from_expression_projection_names = resolveExpressionNode(sort_node.getFillFrom(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - const auto * constant_node = sort_node.getFillFrom()->as(); - if (!constant_node || !isColumnedAsNumber(constant_node->getResultType())) + const auto constant_value = sort_node.getFillFrom()->getConstantValueOrNull(); + if (!constant_value || !isColumnedAsNumber(constant_value->getType())) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "WITH FILL FROM expression must be constant with numeric type. Actual {}. In scope {}", + "Sort FILL FROM expression must be constant with numeric type. Actual {}. In scope {}", sort_node.getFillFrom()->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); + + size_t fill_from_expression_projection_names_size = fill_from_expression_projection_names.size(); + if (fill_from_expression_projection_names_size != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Sort node FILL FROM expression expected 1 projection name. Actual {}", + fill_from_expression_projection_names_size); } + if (sort_node.hasFillTo()) { - resolveExpressionNode(sort_node.getFillTo(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - const auto * constant_node = sort_node.getFillTo()->as(); - if (!constant_node || !isColumnedAsNumber(constant_node->getResultType())) + fill_to_expression_projection_names = resolveExpressionNode(sort_node.getFillTo(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + const auto constant_value = sort_node.getFillTo()->getConstantValueOrNull(); + if (!constant_value || !isColumnedAsNumber(constant_value->getType())) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "WITH FILL TO expression must be constant with numeric type. Actual {}. In scope {}", + "Sort FILL TO expression must be constant with numeric type. Actual {}. In scope {}", sort_node.getFillFrom()->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); + + size_t fill_to_expression_projection_names_size = fill_to_expression_projection_names.size(); + if (fill_to_expression_projection_names_size != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Sort node FILL TO expression expected 1 projection name. Actual {}", + fill_to_expression_projection_names_size); } + if (sort_node.hasFillStep()) { - resolveExpressionNode(sort_node.getFillStep(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - const auto * constant_node = sort_node.getFillStep()->as(); - if (!constant_node) + fill_step_expression_projection_names = resolveExpressionNode(sort_node.getFillStep(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + const auto constant_value = sort_node.getFillStep()->getConstantValueOrNull(); + if (!constant_value) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "WITH FILL TO expression must be constant with numeric or interval type. Actual {}. In scope {}", + "Sort FILL STEP expression must be constant with numeric or interval type. Actual {}. In scope {}", sort_node.getFillStep()->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); - bool is_number = isColumnedAsNumber(constant_node->getResultType()); - bool is_interval = WhichDataType(constant_node->getResultType()).isInterval(); + bool is_number = isColumnedAsNumber(constant_value->getType()); + bool is_interval = WhichDataType(constant_value->getType()).isInterval(); if (!is_number && !is_interval) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "WITH FILL TO expression must be constant with numeric or interval type. Actual {}. In scope {}", + "Sort FILL STEP expression must be constant with numeric or interval type. Actual {}. In scope {}", sort_node.getFillStep()->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); + + size_t fill_step_expression_projection_names_size = fill_step_expression_projection_names.size(); + if (fill_step_expression_projection_names_size != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Sort FILL STEP expression expected 1 projection name. Actual {}", + fill_step_expression_projection_names_size); } + + auto sort_column_projection_name = calculateSortColumnProjectionName(node, + sort_expression_projection_names[0], + fill_from_expression_projection_names.empty() ? "" : fill_from_expression_projection_names.front(), + fill_to_expression_projection_names.empty() ? "" : fill_to_expression_projection_names.front(), + fill_step_expression_projection_names.empty() ? "" : fill_step_expression_projection_names.front()); + + result_projection_names.push_back(std::move(sort_column_projection_name)); + + sort_expression_projection_names.clear(); + fill_from_expression_projection_names.clear(); + fill_to_expression_projection_names.clear(); + fill_step_expression_projection_names.clear(); } + + return result_projection_names; } /** Resolve interpolate columns nodes list. @@ -3898,305 +4538,19 @@ void QueryAnalyzer::resolveWindowNodeList(QueryTreeNodePtr & window_node_list, I resolveWindow(node, scope); } -class SubqueryToProjectionNameMatcher -{ -public: - using Visitor = ConstInDepthQueryTreeVisitor; - - struct Data - { - std::unordered_map & node_to_display_name; - size_t subquery_index = 1; - }; - - static void visit(const QueryTreeNodePtr &, Data &) - { - } - - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node, Data & data) - { - auto child_node_type = child_node->getNodeType(); - if (child_node_type != QueryTreeNodeType::QUERY && child_node_type != QueryTreeNodeType::UNION) - return true; - - data.node_to_display_name.emplace(child_node, "_subquery_" + std::to_string(data.subquery_index)); - ++data.subquery_index; - return false; - } -}; - -using SubqueryToProjectionNameVisitor = SubqueryToProjectionNameMatcher::Visitor; - -String QueryAnalyzer::calculateProjectionNodeDisplayName(QueryTreeNodePtr & node, IdentifierResolveScope & scope) -{ - if (node->hasAlias()) - return node->getAlias(); - - auto calculate_inner_expression_display_name = [&](QueryTreeNodePtr & expression_node, IdentifierResolveScope & node_scope) - { - auto * expression_function_node = expression_node->as(); - if (expression_node->getNodeType() == QueryTreeNodeType::MATCHER) - expression_node = resolveMatcher(expression_node, node_scope); - else if (expression_function_node && expression_function_node->getFunctionName() == "untuple") - resolveFunction(expression_node, node_scope); - - if (expression_node->getNodeType() == QueryTreeNodeType::LIST) - { - auto & list_nodes = expression_node->as().getNodes(); - size_t list_nodes_size = list_nodes.size(); - - WriteBufferFromOwnString result_buffer; - for (size_t i = 0; i < list_nodes_size; ++i) - { - auto & list_expression_node = list_nodes[i]; - String list_node_display_name = calculateProjectionNodeDisplayName(list_expression_node, node_scope); - result_buffer << list_node_display_name; - - if (i + 1 != list_nodes_size) - result_buffer << ", "; - } - - return result_buffer.str(); - } - - return calculateProjectionNodeDisplayName(expression_node, node_scope); - }; - - auto try_to_get_projection_name_from_scope = [&](const QueryTreeNodePtr & expression_node) - { - bool check_in_parent_query = scope.scope_node->getNodeType() != QueryTreeNodeType::QUERY; - IdentifierResolveScope * scope_to_check = &scope; - - while (scope_to_check != nullptr) - { - auto projection_name_it = scope_to_check->node_to_projection_name.find(expression_node); - if (projection_name_it != scope_to_check->node_to_projection_name.end()) - return projection_name_it->second; - - scope_to_check = scope_to_check->parent_scope; - if (scope_to_check && scope_to_check->scope_node->getNodeType() == QueryTreeNodeType::QUERY) - { - if (check_in_parent_query) - check_in_parent_query = false; - else - break; - } - } - - return String(); - }; - - auto projection_name_from_scope = try_to_get_projection_name_from_scope(node); - if (!projection_name_from_scope.empty()) - return projection_name_from_scope; - - auto node_type = node->getNodeType(); - if (node_type == QueryTreeNodeType::CONSTANT) - { - const auto & constant_node = node->as(); - return constant_node.getName(); - } - else if (node_type == QueryTreeNodeType::COLUMN) - { - const auto & column_node = node->as(); - const auto & column_name = column_node.getColumnName(); - return column_name; - } - else if (node_type == QueryTreeNodeType::IDENTIFIER) - { - const auto & identifier_node = node->as(); - auto identifier = identifier_node.getIdentifier(); - IdentifierLookup lookup {identifier, IdentifierLookupContext::EXPRESSION}; - auto resolved_identifier_result = tryResolveIdentifier(lookup, scope); - - if (resolved_identifier_result.resolved_identifier && resolved_identifier_result.isResolvedFromJoinTree()) - { - projection_name_from_scope = try_to_get_projection_name_from_scope(resolved_identifier_result.resolved_identifier); - - if (!projection_name_from_scope.empty()) - return projection_name_from_scope; - - if (auto * column_node = resolved_identifier_result.resolved_identifier->as()) - return column_node->getColumnName(); - } - - return identifier_node.getIdentifier().getFullName(); - } - else if (node_type == QueryTreeNodeType::MATCHER) - { - /// Top level matcher - return {}; - } - else if (node_type == QueryTreeNodeType::FUNCTION) - { - auto & function_node = node->as(); - - WriteBufferFromOwnString buffer; - buffer << function_node.getFunctionName(); - - auto & function_parameters_nodes = function_node.getParameters().getNodes(); - - if (!function_parameters_nodes.empty()) - { - buffer << '('; - - size_t function_parameters_nodes_size = function_parameters_nodes.size(); - for (size_t i = 0; i < function_parameters_nodes_size; ++i) - { - auto & function_parameter_node = function_parameters_nodes[i]; - String function_parameter_node_display_name = calculate_inner_expression_display_name(function_parameter_node, scope); - buffer << function_parameter_node_display_name; - - if (i + 1 != function_parameters_nodes_size) - buffer << ", "; - } - - buffer << ')'; - } - - buffer << '('; - - auto & function_arguments_nodes = function_node.getArguments().getNodes(); - size_t function_arguments_nodes_size = function_arguments_nodes.size(); - for (size_t i = 0; i < function_arguments_nodes_size; ++i) - { - auto & function_argument_node = function_arguments_nodes[i]; - String function_argument_node_display_name = calculate_inner_expression_display_name(function_argument_node, scope); - buffer << function_argument_node_display_name; - - if (i + 1 != function_arguments_nodes_size) - buffer << ", "; - } - - buffer << ')'; - - return buffer.str(); - } - else if (node_type == QueryTreeNodeType::LAMBDA) - { - auto & lambda_node = node->as(); - IdentifierResolveScope lambda_scope(node, &scope /*parent_scope*/); - lambda_scope.projection_names_calculation_stage = true; - String lambda_expression_display_name = calculate_inner_expression_display_name(lambda_node.getExpression(), lambda_scope); - - WriteBufferFromOwnString buffer; - buffer << "lambda(tuple("; - - const auto & lambda_argument_names = lambda_node.getArgumentNames(); - size_t lambda_argument_names_size = lambda_argument_names.size(); - - for (size_t i = 0; i < lambda_argument_names_size; ++i) - { - const auto & argument_name = lambda_argument_names[i]; - buffer << argument_name; - - if (i + 1 != lambda_argument_names_size) - buffer << ", "; - } - - buffer << "), "; - buffer << lambda_expression_display_name; - buffer << ')'; - - return buffer.str(); - } - else if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Subquery name must be already collected"); - } - else if (node_type == QueryTreeNodeType::TABLE) - { - /// Table node can be second argument of IN function - const auto & table_node = node->as(); - return table_node.getStorageID().getFullNameNotQuoted(); - } - else if (node_type == QueryTreeNodeType::TABLE_FUNCTION) - { - /// Table function node can be second argument of IN function - auto & table_function_node = node->as(); - - WriteBufferFromOwnString buffer; - buffer << table_function_node.getTableFunctionName(); - - buffer << '('; - - auto & table_function_arguments_nodes = table_function_node.getArguments().getNodes(); - size_t table_function_arguments_nodes_size = table_function_arguments_nodes.size(); - for (size_t i = 0; i < table_function_arguments_nodes_size; ++i) - { - auto & function_argument_node = table_function_arguments_nodes[i]; - if (auto * identifier_node = function_argument_node->as()) - { - if (identifier_node->hasAlias()) - buffer << identifier_node->getAlias(); - else - buffer << identifier_node->getIdentifier().getFullName(); - } - else - { - buffer << calculateProjectionNodeDisplayName(function_argument_node, scope); - } - - if (i + 1 != table_function_arguments_nodes_size) - buffer << ", "; - } - - buffer << ')'; - - return buffer.str(); - } - else - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Invalid projection node {} {}", node->getNodeType(), node->dumpTree()); - } -} - NamesAndTypes QueryAnalyzer::resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope) { - auto & initial_node_list = projection_node_list->as(); + ProjectionNames projection_names = resolveExpressionNodeList(projection_node_list, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - scope.projection_names_calculation_stage = true; + auto projection_nodes = projection_node_list->as().getNodes(); + size_t projection_nodes_size = projection_nodes.size(); - SubqueryToProjectionNameVisitor::Data subquery_to_projection_name_visitor_data {scope.node_to_projection_name}; - SubqueryToProjectionNameVisitor subquery_to_projection_name_visitor(subquery_to_projection_name_visitor_data); - subquery_to_projection_name_visitor.visit(projection_node_list); - - auto initial_node_list_nodes_copy = initial_node_list.getNodes(); - size_t list_nodes_copy_size = initial_node_list_nodes_copy.size(); - - std::vector> projection_nodes_with_display_name; - projection_nodes_with_display_name.reserve(list_nodes_copy_size); - - for (size_t i = 0; i < list_nodes_copy_size; ++i) - { - auto & node = initial_node_list_nodes_copy[i]; - - String display_name = calculateProjectionNodeDisplayName(node, scope); - - auto node_to_resolve = node; - resolveExpressionNode(node_to_resolve, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - - if (auto * expression_list = node_to_resolve->as()) - { - for (auto & expression_list_node : expression_list->getNodes()) - projection_nodes_with_display_name.emplace_back(expression_list_node, calculateProjectionNodeDisplayName(expression_list_node, scope)); - } - else - { - projection_nodes_with_display_name.emplace_back(node_to_resolve, std::move(display_name)); - } - } - - size_t projection_nodes_size = projection_nodes_with_display_name.size(); - - auto projection_result_node_list = std::make_shared(); NamesAndTypes projection_columns; projection_columns.reserve(projection_nodes_size); - projection_result_node_list->getNodes().reserve(projection_nodes_size); for (size_t i = 0; i < projection_nodes_size; ++i) { - auto && projection_node = projection_nodes_with_display_name[i].first; + auto projection_node = projection_nodes[i]; if (projection_node->getNodeType() != QueryTreeNodeType::CONSTANT && projection_node->getNodeType() != QueryTreeNodeType::FUNCTION && @@ -4206,14 +4560,9 @@ NamesAndTypes QueryAnalyzer::resolveProjectionExpressionNodeList(QueryTreeNodePt throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Projection node must be constant, function, column, query or union"); - projection_columns.emplace_back(projection_nodes_with_display_name[i].second, projection_node->getResultType()); - projection_result_node_list->getNodes().emplace_back(std::move(projection_node)); + projection_columns.emplace_back(projection_names[i], projection_node->getResultType()); } - scope.projection_names_calculation_stage = false; - scope.node_to_projection_name.clear(); - - projection_node_list = std::move(projection_result_node_list); return projection_columns; } @@ -4240,7 +4589,7 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod case QueryTreeNodeType::IDENTIFIER: { auto & from_table_identifier = current_join_tree_node->as(); - auto table_identifier_lookup = IdentifierLookup{from_table_identifier.getIdentifier(), IdentifierLookupContext::TABLE}; + auto table_identifier_lookup = IdentifierLookup{from_table_identifier.getIdentifier(), IdentifierLookupContext::TABLE_EXPRESSION}; IdentifierResolveSettings resolve_settings; /// In join tree initialization ignore join tree as identifier lookup source @@ -4252,7 +4601,7 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod if (!resolved_identifier) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown {} identifier {} in scope {}", - toStringLowercase(IdentifierLookupContext::TABLE), + toStringLowercase(IdentifierLookupContext::TABLE_EXPRESSION), from_table_identifier.getIdentifier().getFullName(), scope.scope_node->formatASTForErrorMessage()); @@ -5053,7 +5402,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (settings.enable_positional_arguments) replaceNodesWithPositionalArguments(query_node_typed.getOrderByNode(), query_node_typed.getProjection().getNodes(), scope); - resolveSortColumnsNodeList(query_node_typed.getOrderByNode(), scope); + resolveSortNodeList(query_node_typed.getOrderByNode(), scope); } if (query_node_typed.hasInterpolate()) diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index dcababb6296..c144cad7b83 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -134,13 +134,22 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s if (hasAlias()) buffer << ", alias: " << getAlias(); - buffer << ", is_subquery: " << is_subquery; - buffer << ", is_cte: " << is_cte; - buffer << ", is_distinct: " << is_distinct; - buffer << ", is_limit_with_ties: " << is_limit_with_ties; - buffer << ", is_group_by_with_totals: " << is_group_by_with_totals; + if (is_subquery) + buffer << ", is_subquery: " << is_subquery; - std::string group_by_type = "ordinary"; + if (is_cte) + buffer << ", is_cte: " << is_cte; + + if (is_distinct) + buffer << ", is_distinct: " << is_distinct; + + if (is_limit_with_ties) + buffer << ", is_limit_with_ties: " << is_limit_with_ties; + + if (is_group_by_with_totals) + buffer << ", is_group_by_with_totals: " << is_group_by_with_totals; + + std::string group_by_type; if (is_group_by_with_rollup) group_by_type = "rollup"; else if (is_group_by_with_cube) @@ -148,7 +157,8 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s else if (is_group_by_with_grouping_sets) group_by_type = "grouping_sets"; - buffer << ", group_by_type: " << group_by_type; + if (!group_by_type.empty()) + buffer << ", group_by_type: " << group_by_type; if (!cte_name.empty()) buffer << ", cte_name: " << cte_name; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 531fe0ae5d8..f99e87f08d9 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -60,6 +60,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int EXPECTED_ALL_OR_ANY; extern const int NOT_IMPLEMENTED; + extern const int BAD_ARGUMENTS; } namespace diff --git a/src/Analyzer/WindowNode.cpp b/src/Analyzer/WindowNode.cpp index c4d11153a31..94fa8f83505 100644 --- a/src/Analyzer/WindowNode.cpp +++ b/src/Analyzer/WindowNode.cpp @@ -94,7 +94,9 @@ void WindowNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, if (hasAlias()) buffer << ", alias: " << getAlias(); - buffer << ", parent_window_name: " << parent_window_name; + if (!parent_window_name.empty()) + buffer << ", parent_window_name: " << parent_window_name; + buffer << ", frame_type: " << window_frame.type; auto window_frame_bound_type_to_string = [](WindowFrame::BoundaryType boundary_type, bool boundary_preceding) diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 576bf9b861b..92cd63999b5 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -84,7 +84,7 @@ public: else { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Function {} is supported only if second argument is constant or table expression", + "Function '{}' is supported only if second argument is constant or table expression", function_node->getFunctionName()); } } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 1597c7a5e95..a3fb81d707a 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -75,12 +75,11 @@ namespace ErrorCodes /** ClickHouse query planner. * * TODO: Support JOIN with JOIN engine. + * TODO: Support VIEWs. * TODO: JOIN drop unnecessary columns after ON, USING section - * TODO: Support display names * TODO: Support RBAC. Support RBAC for ALIAS columns * TODO: Support distributed query processing * TODO: Support PREWHERE - * TODO: Support ORDER BY * TODO: Support DISTINCT * TODO: Support trivial count optimization * TODO: Support projections @@ -89,7 +88,7 @@ namespace ErrorCodes * TODO: Support max streams * TODO: Support ORDER BY read in order optimization * TODO: Support GROUP BY read in order optimization - * TODO: Support Key Condition + * TODO: Support Key Condition. Support indexes for IN function. */ namespace diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.reference b/tests/queries/0_stateless/02378_analyzer_projection_names.reference index 179b125cc08..ea16b2f127b 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.reference +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.reference @@ -133,12 +133,36 @@ plus(a, id) UInt64 plus(id, b) UInt64 plus(b, b) UInt64 plus(id, id) UInt64 -SELECT '--Fix'; ---Fix +SELECT '--'; +-- DESCRIBE (SELECT test_table.* REPLACE id + (id AS id_alias) AS id, id_alias FROM test_table); -plus(id, id) UInt64 +plus(id, id_alias) UInt64 value String id_alias UInt64 +SELECT 'Matcher'; +Matcher +DESCRIBE (SELECT * FROM test_table); +id UInt64 +value String +SELECT '--'; +-- +DESCRIBE (SELECT test_table.* FROM test_table); +id UInt64 +value String +SELECT '--'; +-- +DESCRIBE (SELECT 1 AS id, 2 AS value, * FROM test_table); +id UInt8 +value UInt8 +test_table.id UInt64 +test_table.value String +SELECT '--'; +-- +DESCRIBE (SELECT 1 AS id, 2 AS value, * FROM test_table AS t1); +id UInt8 +value UInt8 +t1.id UInt64 +t1.value String SELECT 'Lambda'; Lambda DESCRIBE (SELECT arrayMap(x -> x + 1, [1,2,3])); @@ -227,7 +251,7 @@ SELECT '--'; DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, untuple(a) AS untupled_value, arrayMap(x -> untupled_value, [1,2,3]) FROM test_table); a Tuple(id UInt64) untupled_value.id UInt64 -arrayMap(lambda(tuple(x), untupled_value), [1, 2, 3]) Array(UInt64) +arrayMap(lambda(tuple(x), untupled_value.id), [1, 2, 3]) Array(UInt64) SELECT '--'; -- DESCRIBE (SELECT cast(tuple(1), 'Tuple (id UInt64)') AS a, untuple(a) AS untupled_value, arrayMap(x -> untupled_value AS untupled_value_in_lambda, [1,2,3]) FROM test_table); @@ -238,6 +262,13 @@ SELECT 'Standalone lambda'; Standalone lambda DESCRIBE (WITH x -> x + 1 AS test_lambda SELECT test_lambda(1)); test_lambda(1) UInt16 +SELECT '--'; +-- +DESCRIBE (WITH x -> * AS test_lambda SELECT test_lambda(1) AS value, value FROM test_table); +id UInt64 +value String +id UInt64 +value String SELECT 'Subquery'; Subquery DESCRIBE (SELECT (SELECT 1), (SELECT 2), (SELECT 3) AS a, (SELECT 4)); @@ -257,6 +288,118 @@ DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b) AS c, c.a, c.b); c Tuple(a UInt8, b UInt8) c.a UInt8 c.b UInt8 +SELECT 'Window functions'; +Window functions +DESCRIBE (SELECT count() OVER ()); +count() OVER () UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER () AS window_function); +window_function UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (PARTITION BY id) FROM test_table); +count() OVER (PARTITION BY id) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (PARTITION BY id, value) FROM test_table); +count() OVER (PARTITION BY id, value) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id) FROM test_table); +count() OVER (PARTITION BY id, value ORDER BY id ASC) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS CURRENT ROW) FROM test_table); +count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN CURRENT ROW AND CURRENT ROW) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN CURRENT ROW AND CURRENT ROW) FROM test_table); +count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN CURRENT ROW AND CURRENT ROW) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC RANGE CURRENT ROW) FROM test_table); +count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC RANGE BETWEEN CURRENT ROW AND CURRENT ROW) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC RANGE BETWEEN CURRENT ROW AND CURRENT ROW) FROM test_table); +count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC RANGE BETWEEN CURRENT ROW AND CURRENT ROW) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (PARTITION BY (id AS id_alias), (value AS value_alias) ORDER BY id ASC, value DESC ROWS CURRENT ROW) FROM test_table); +count() OVER (PARTITION BY id_alias, value_alias ORDER BY id ASC, value DESC ROWS BETWEEN CURRENT ROW AND CURRENT ROW) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY (id AS id_alias) ASC, (value AS value_alias) DESC ROWS CURRENT ROW) FROM test_table); +count() OVER (PARTITION BY id, value ORDER BY id_alias ASC, value_alias DESC ROWS BETWEEN CURRENT ROW AND CURRENT ROW) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN 1 PRECEDING AND 2 FOLLOWING) FROM test_table); +count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN 1 PRECEDING AND 2 FOLLOWING) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN 1 + 1 PRECEDING AND 2 + 2 FOLLOWING) FROM test_table); +count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN plus(1, 1) PRECEDING AND plus(2, 2) FOLLOWING) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN ((1 + 1) AS frame_offset_begin) PRECEDING AND ((2 + 2) AS frame_offset_end) FOLLOWING) FROM test_table); +count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN frame_offset_begin PRECEDING AND frame_offset_end FOLLOWING) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (ORDER BY toNullable(id) NULLS FIRST) FROM test_table); +count() OVER (ORDER BY toNullable(id) ASC NULLS FIRST) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (ORDER BY toNullable(id) NULLS LAST) FROM test_table); +count() OVER (ORDER BY toNullable(id) ASC NULLS LAST) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (ORDER BY value COLLATE 'EN') FROM test_table); +count() OVER (ORDER BY value ASC COLLATE en) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (ORDER BY id WITH FILL FROM 1 TO 5 STEP 1) FROM test_table); +count() OVER (ORDER BY id ASC WITH FILL FROM 1 TO 5 STEP 1) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (ORDER BY id WITH FILL FROM 1 + 1 TO 6 STEP 1 + 1) FROM test_table); +count() OVER (ORDER BY id ASC WITH FILL FROM plus(1, 1) TO 6 STEP plus(1, 1)) UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (ORDER BY id WITH FILL FROM ((1 + 1) AS from) TO (6 AS to) STEP ((1 + 1) AS step)) FROM test_table); +count() OVER (ORDER BY id ASC WITH FILL FROM from TO to STEP step) UInt64 +SELECT 'Window functions WINDOW'; +Window functions WINDOW +DESCRIBE (SELECT count() OVER window_name FROM test_table WINDOW window_name AS (PARTITION BY id)); +count() OVER window_name UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER window_name FROM test_table WINDOW window_name AS (PARTITION BY id ORDER BY value)); +count() OVER window_name UInt64 +SELECT '--'; +-- +DESCRIBE (SELECT count() OVER (window_name ORDER BY id) FROM test_table WINDOW window_name AS (PARTITION BY id)); +count() OVER (window_name ORDER BY id ASC) UInt64 +SELECT 'IN function'; +IN function +DESCRIBE (SELECT id IN (SELECT 1) FROM test_table); +in(id, _subquery_1) UInt8 +SELECT '--'; +-- +DESCRIBE (SELECT id IN (SELECT id FROM test_table_in) FROM test_table); +in(id, _subquery_1) UInt8 +SELECT '--'; +-- +DESCRIBE (SELECT id IN test_table_in FROM test_table); +in(id, test_table_in) UInt8 +SELECT '--'; +-- +DESCRIBE (WITH test_table_in_cte AS (SELECT id FROM test_table) SELECT id IN (SELECT id FROM test_table_in_cte) FROM test_table); +in(id, _subquery_1) UInt8 +SELECT '--'; +-- +DESCRIBE (WITH test_table_in_cte AS (SELECT id FROM test_table) SELECT id IN test_table_in_cte FROM test_table); +in(id, test_table_in_cte) UInt8 SELECT 'Joins'; Joins DESCRIBE (SELECT * FROM test_table_join_1, test_table_join_2); @@ -476,3 +619,78 @@ value_join_2 String t3.id UInt64 t3.value String value_join_3 String +SELECT 'Joins USING'; +Joins USING +DESCRIBE (SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id)); +id UInt64 +t1.value String +value_join_1 String +t2.value String +value_join_2 String +SELECT '--'; +-- +DESCRIBE (SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id, value)); +id UInt64 +value String +value_join_1 String +value_join_2 String +SELECT '--'; +-- +DESCRIBE (SELECT id, t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id)); +id UInt64 +t1.id UInt64 +t1.value String +t2.id UInt64 +t2.value String +SELECT '--'; +-- +DESCRIBE (SELECT id, value, t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id, value)); +id UInt64 +value String +t1.id UInt64 +t1.value String +t2.id UInt64 +t2.value String +SELECT 'Multiple Joins USING'; +Multiple Joins USING +SELECT '--'; +-- +DESCRIBE (SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING (id)); +id UInt64 +t1.value String +value_join_1 String +t2.value String +value_join_2 String +t3.value String +value_join_3 String +SELECT '--'; +-- +DESCRIBE (SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id, value) INNER JOIN test_table_join_3 AS t3 USING (id, value)); +id UInt64 +value String +value_join_1 String +value_join_2 String +value_join_3 String +SELECT '--'; +-- +DESCRIBE (SELECT id, t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING (id)); +id UInt64 +t1.id UInt64 +t1.value String +t2.id UInt64 +t2.value String +t3.id UInt64 +t3.value String +SELECT '--'; +-- +DESCRIBE (SELECT id, value, t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id, value) INNER JOIN test_table_join_3 AS t3 USING (id, value)); +id UInt64 +value String +t1.id UInt64 +t1.value String +t2.id UInt64 +t2.value String +t3.id UInt64 +t3.value String diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.sql b/tests/queries/0_stateless/02378_analyzer_projection_names.sql index c17858cccf6..5703f63deea 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.sql +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.sql @@ -9,6 +9,12 @@ CREATE TABLE test_table INSERT INTO test_table VALUES (0, 'Value'); +DROP TABLE IF EXISTS test_table_in; +CREATE TABLE test_table_in +( + id UInt64 +) ENGINE=TinyLog; + DROP TABLE IF EXISTS test_table_compound; CREATE TABLE test_table_compound ( @@ -142,10 +148,26 @@ SELECT '--'; DESCRIBE (SELECT plus(test_table.id AS a, test_table.id), plus(id, id AS b), plus(b, b), plus(test_table.id, test_table.id) FROM test_table); -SELECT '--Fix'; +SELECT '--'; DESCRIBE (SELECT test_table.* REPLACE id + (id AS id_alias) AS id, id_alias FROM test_table); +SELECT 'Matcher'; + +DESCRIBE (SELECT * FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT test_table.* FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT 1 AS id, 2 AS value, * FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT 1 AS id, 2 AS value, * FROM test_table AS t1); + SELECT 'Lambda'; DESCRIBE (SELECT arrayMap(x -> x + 1, [1,2,3])); @@ -230,6 +252,10 @@ SELECT 'Standalone lambda'; DESCRIBE (WITH x -> x + 1 AS test_lambda SELECT test_lambda(1)); +SELECT '--'; + +DESCRIBE (WITH x -> * AS test_lambda SELECT test_lambda(1) AS value, value FROM test_table); + SELECT 'Subquery'; DESCRIBE (SELECT (SELECT 1), (SELECT 2), (SELECT 3) AS a, (SELECT 4)); @@ -242,6 +268,118 @@ SELECT '--'; DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b) AS c, c.a, c.b); +SELECT 'Window functions'; + +DESCRIBE (SELECT count() OVER ()); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER () AS window_function); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (PARTITION BY id) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (PARTITION BY id, value) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS CURRENT ROW) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN CURRENT ROW AND CURRENT ROW) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC RANGE CURRENT ROW) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC RANGE BETWEEN CURRENT ROW AND CURRENT ROW) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (PARTITION BY (id AS id_alias), (value AS value_alias) ORDER BY id ASC, value DESC ROWS CURRENT ROW) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY (id AS id_alias) ASC, (value AS value_alias) DESC ROWS CURRENT ROW) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN 1 PRECEDING AND 2 FOLLOWING) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN 1 + 1 PRECEDING AND 2 + 2 FOLLOWING) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN ((1 + 1) AS frame_offset_begin) PRECEDING AND ((2 + 2) AS frame_offset_end) FOLLOWING) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (ORDER BY toNullable(id) NULLS FIRST) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (ORDER BY toNullable(id) NULLS LAST) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (ORDER BY value COLLATE 'EN') FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (ORDER BY id WITH FILL FROM 1 TO 5 STEP 1) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (ORDER BY id WITH FILL FROM 1 + 1 TO 6 STEP 1 + 1) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (ORDER BY id WITH FILL FROM ((1 + 1) AS from) TO (6 AS to) STEP ((1 + 1) AS step)) FROM test_table); + +SELECT 'Window functions WINDOW'; + +DESCRIBE (SELECT count() OVER window_name FROM test_table WINDOW window_name AS (PARTITION BY id)); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER window_name FROM test_table WINDOW window_name AS (PARTITION BY id ORDER BY value)); + +SELECT '--'; + +DESCRIBE (SELECT count() OVER (window_name ORDER BY id) FROM test_table WINDOW window_name AS (PARTITION BY id)); + +SELECT 'IN function'; + +DESCRIBE (SELECT id IN (SELECT 1) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT id IN (SELECT id FROM test_table_in) FROM test_table); + +SELECT '--'; + +DESCRIBE (SELECT id IN test_table_in FROM test_table); + +SELECT '--'; + +DESCRIBE (WITH test_table_in_cte AS (SELECT id FROM test_table) SELECT id IN (SELECT id FROM test_table_in_cte) FROM test_table); + +SELECT '--'; + +DESCRIBE (WITH test_table_in_cte AS (SELECT id FROM test_table) SELECT id IN test_table_in_cte FROM test_table); + SELECT 'Joins'; DESCRIBE (SELECT * FROM test_table_join_1, test_table_join_2); @@ -331,9 +469,46 @@ SELECT '--'; DESCRIBE (SELECT t1.id, t1.value, t1.value_join_1, t2.id, t2.value, t2.value_join_2, t3.id, t3.value, t3.value_join_3 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id); +SELECT 'Joins USING'; + +DESCRIBE (SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id)); + +SELECT '--'; + +DESCRIBE (SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id, value)); + +SELECT '--'; + +DESCRIBE (SELECT id, t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id)); + +SELECT '--'; + +DESCRIBE (SELECT id, value, t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id, value)); + +SELECT 'Multiple Joins USING'; + +SELECT '--'; + +DESCRIBE (SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING (id)); + +SELECT '--'; + +DESCRIBE (SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id, value) INNER JOIN test_table_join_3 AS t3 USING (id, value)); + +SELECT '--'; + +DESCRIBE (SELECT id, t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING (id)); + +SELECT '--'; + +DESCRIBE (SELECT id, value, t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id, value) INNER JOIN test_table_join_3 AS t3 USING (id, value)); + -- { echoOff } DROP TABLE test_table_join_1; DROP TABLE test_table_join_2; +DROP TABLE test_table_join_3; DROP TABLE test_table; DROP TABLE test_table_compound; From 49a129008ba9015fa7e92b8a000141c50e6c1220 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 7 Oct 2022 12:44:28 +0200 Subject: [PATCH 126/188] Fixed code review issues --- src/Analyzer/ArrayJoinNode.cpp | 7 ++- src/Analyzer/ArrayJoinNode.h | 16 ++++-- src/Analyzer/ColumnNode.cpp | 17 +++---- src/Analyzer/ColumnTransformers.cpp | 35 +++++++++++--- src/Analyzer/ColumnTransformers.h | 43 ++++++++--------- src/Analyzer/ConstantNode.cpp | 3 +- src/Analyzer/ConstantNode.h | 2 + src/Analyzer/FunctionNode.cpp | 8 +-- src/Analyzer/FunctionNode.h | 16 +++--- src/Analyzer/IQueryTreeNode.cpp | 5 ++ src/Analyzer/IQueryTreeNode.h | 2 + src/Analyzer/IdentifierNode.cpp | 11 +++++ src/Analyzer/IdentifierNode.h | 13 ++--- src/Analyzer/InterpolateNode.cpp | 2 +- src/Analyzer/InterpolateNode.h | 8 +-- src/Analyzer/JoinNode.cpp | 23 +++------ src/Analyzer/JoinNode.h | 17 ++++--- src/Analyzer/LambdaNode.cpp | 11 ++--- src/Analyzer/LambdaNode.h | 7 ++- src/Analyzer/ListNode.cpp | 5 ++ src/Analyzer/ListNode.h | 2 +- src/Analyzer/MatcherNode.cpp | 51 ++++++++++++++------ src/Analyzer/MatcherNode.h | 1 + src/Analyzer/QueryNode.cpp | 2 +- src/Analyzer/SortNode.cpp | 4 +- src/Analyzer/TableFunctionNode.cpp | 4 +- src/Analyzer/TableFunctionNode.h | 5 +- src/Analyzer/TableNode.cpp | 20 ++++---- src/Analyzer/TableNode.h | 15 ++---- src/Analyzer/UnionNode.cpp | 2 +- src/Analyzer/Utils.cpp | 20 ++++++-- src/Analyzer/WindowNode.cpp | 4 +- src/Analyzer/tests/gtest_query_tree_node.cpp | 2 + 33 files changed, 223 insertions(+), 160 deletions(-) diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp index 20eb3f7f074..1504a7ab7a2 100644 --- a/src/Analyzer/ArrayJoinNode.cpp +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -12,9 +12,9 @@ namespace DB { ArrayJoinNode::ArrayJoinNode(QueryTreeNodePtr table_expression_, QueryTreeNodePtr join_expressions_, bool is_left_) - : is_left(is_left_) + : IQueryTreeNode(children_size) + , is_left(is_left_) { - children.resize(children_size); children[table_expression_child_index] = std::move(table_expression_); children[join_expressions_child_index] = std::move(join_expressions_); } @@ -65,8 +65,7 @@ ASTPtr ArrayJoinNode::toASTImpl() const QueryTreeNodePtr ArrayJoinNode::cloneImpl() const { - ArrayJoinNodePtr result_array_join_node(new ArrayJoinNode(is_left)); - return result_array_join_node; + return std::make_shared(getTableExpression(), getJoinExpressionsNode(), is_left); } } diff --git a/src/Analyzer/ArrayJoinNode.h b/src/Analyzer/ArrayJoinNode.h index 248e8443143..d2e0cd6e6eb 100644 --- a/src/Analyzer/ArrayJoinNode.h +++ b/src/Analyzer/ArrayJoinNode.h @@ -15,10 +15,20 @@ namespace DB /** Array join node represents array join in query tree. * - * In query tree join expression is represented by list query tree node. + * In query tree array join expressions are represented by list query tree node. * * Example: SELECT id FROM test_table ARRAY JOIN [1, 2, 3] as a. + * + * Multiple expressions can be inside single array join. * Example: SELECT id FROM test_table ARRAY JOIN [1, 2, 3] as a, [4, 5, 6] as b. + * Example: SELECT id FROM test_table ARRAY JOIN array_column_1 AS value_1, array_column_2 AS value_2. + * + * Multiple array joins can be inside JOIN TREE. + * Example: SELECT id FROM test_table ARRAY JOIN array_column_1 ARRAY JOIN array_column_2. + * + * Array join can be used inside JOIN TREE with ordinary JOINS. + * Example: SELECT t1.id FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id ARRAY JOIN [1,2,3]; + * Example: SELECT t1.id FROM test_table_1 AS t1 ARRAY JOIN [1,2,3] INNER JOIN test_table_2 AS t2 ON t1.id = t2.id; */ class ArrayJoinNode; using ArrayJoinNodePtr = std::shared_ptr; @@ -90,10 +100,6 @@ protected: QueryTreeNodePtr cloneImpl() const override; private: - explicit ArrayJoinNode(bool is_left_) - : is_left(is_left_) - {} - bool is_left = false; static constexpr size_t table_expression_child_index = 0; diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index 4dd3f85fa99..231acb695cd 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -17,17 +17,17 @@ namespace ErrorCodes } ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_) - : column(std::move(column_)) + : IQueryTreeNode(children_size) + , column(std::move(column_)) , column_source(std::move(column_source_)) { - children.resize(children_size); } ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_) - : column(std::move(column_)) + : IQueryTreeNode(children_size) + , column(std::move(column_)) , column_source(std::move(column_source_)) { - children.resize(children_size); children[expression_child_index] = std::move(expression_node_); } @@ -45,11 +45,7 @@ QueryTreeNodePtr ColumnNode::getColumnSource() const QueryTreeNodePtr ColumnNode::getColumnSourceOrNull() const { - auto lock = column_source.lock(); - if (!lock) - return nullptr; - - return lock; + return column_source.lock(); } void ColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t indent) const @@ -109,8 +105,7 @@ void ColumnNode::updateTreeHashImpl(HashState & hash_state) const QueryTreeNodePtr ColumnNode::cloneImpl() const { - auto clone_result = std::make_shared(column, column_source); - return clone_result; + return std::make_shared(column, column_source); } void ColumnNode::getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update) diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp index 6568d82afc0..541d116bc59 100644 --- a/src/Analyzer/ColumnTransformers.cpp +++ b/src/Analyzer/ColumnTransformers.cpp @@ -21,6 +21,8 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +/// IColumnTransformerNode implementation + const char * toString(ColumnTransfomerType type) { switch (type) @@ -31,6 +33,10 @@ const char * toString(ColumnTransfomerType type) } } +IColumnTransformerNode::IColumnTransformerNode(size_t children_size) + : IQueryTreeNode(children_size) +{} + /// ApplyColumnTransformerNode implementation const char * toString(ApplyColumnTransformerType type) @@ -43,6 +49,7 @@ const char * toString(ApplyColumnTransformerType type) } ApplyColumnTransformerNode::ApplyColumnTransformerNode(QueryTreeNodePtr expression_node_) + : IColumnTransformerNode(children_size) { if (expression_node_->getNodeType() == QueryTreeNodeType::LAMBDA) apply_transformer_type = ApplyColumnTransformerType::LAMBDA; @@ -53,7 +60,6 @@ ApplyColumnTransformerNode::ApplyColumnTransformerNode(QueryTreeNodePtr expressi "Apply column transformer expression must be lambda or function. Actual {}", expression_node_->getNodeTypeName()); - children.resize(1); children[expression_child_index] = std::move(expression_node_); } @@ -111,6 +117,21 @@ QueryTreeNodePtr ApplyColumnTransformerNode::cloneImpl() const /// ExceptColumnTransformerNode implementation +ExceptColumnTransformerNode::ExceptColumnTransformerNode(Names except_column_names_, bool is_strict_) + : IColumnTransformerNode(children_size) + , except_transformer_type(ExceptColumnTransformerType::COLUMN_LIST) + , except_column_names(std::move(except_column_names_)) + , is_strict(is_strict_) +{ +} + +ExceptColumnTransformerNode::ExceptColumnTransformerNode(std::shared_ptr column_matcher_) + : IColumnTransformerNode(children_size) + , except_transformer_type(ExceptColumnTransformerType::REGEXP) + , column_matcher(std::move(column_matcher_)) +{ +} + bool ExceptColumnTransformerNode::isColumnMatching(const std::string & column_name) const { if (column_matcher) @@ -226,9 +247,9 @@ QueryTreeNodePtr ExceptColumnTransformerNode::cloneImpl() const /// ReplaceColumnTransformerNode implementation ReplaceColumnTransformerNode::ReplaceColumnTransformerNode(const std::vector & replacements_, bool is_strict_) - : is_strict(is_strict_) + : IColumnTransformerNode(children_size) + , is_strict(is_strict_) { - children.resize(1); children[replacements_child_index] = std::make_shared(); auto & replacement_expressions_nodes = getReplacements().getNodes(); @@ -326,12 +347,12 @@ ASTPtr ReplaceColumnTransformerNode::toASTImpl() const QueryTreeNodePtr ReplaceColumnTransformerNode::cloneImpl() const { - ReplaceColumnTransformerNodePtr result_replace_transformers(new ReplaceColumnTransformerNode()); + auto result_replace_transformer = std::make_shared(std::vector{}, false); - result_replace_transformers->is_strict = is_strict; - result_replace_transformers->replacements_names = replacements_names; + result_replace_transformer->is_strict = is_strict; + result_replace_transformer->replacements_names = replacements_names; - return result_replace_transformers; + return result_replace_transformer; } } diff --git a/src/Analyzer/ColumnTransformers.h b/src/Analyzer/ColumnTransformers.h index f788b6fe5e6..221888160a5 100644 --- a/src/Analyzer/ColumnTransformers.h +++ b/src/Analyzer/ColumnTransformers.h @@ -75,7 +75,6 @@ using ColumnTransformersNodes = std::vector; class IColumnTransformerNode : public IQueryTreeNode { public: - /// Get transformer type virtual ColumnTransfomerType getTransformerType() const = 0; @@ -89,6 +88,10 @@ public: { return QueryTreeNodeType::TRANSFORMER; } + +protected: + /// Construct column transformer node and resize children to children size + explicit IColumnTransformerNode(size_t children_size); }; enum class ApplyColumnTransformerType @@ -142,7 +145,9 @@ protected: private: ApplyColumnTransformerType apply_transformer_type = ApplyColumnTransformerType::LAMBDA; + static constexpr size_t expression_child_index = 0; + static constexpr size_t children_size = expression_child_index + 1; }; /// Except column transformer type @@ -171,20 +176,10 @@ class ExceptColumnTransformerNode final : public IColumnTransformerNode { public: /// Initialize except column transformer with column names - explicit ExceptColumnTransformerNode(Names except_column_names_, bool is_strict_) - : except_transformer_type(ExceptColumnTransformerType::COLUMN_LIST) - , is_strict(is_strict_) - , except_column_names(std::move(except_column_names_)) - { - } + explicit ExceptColumnTransformerNode(Names except_column_names_, bool is_strict_); /// Initialize except column transformer with regexp column matcher - explicit ExceptColumnTransformerNode(std::shared_ptr column_matcher_) - : except_transformer_type(ExceptColumnTransformerType::REGEXP) - , is_strict(false) - , column_matcher(std::move(column_matcher_)) - { - } + explicit ExceptColumnTransformerNode(std::shared_ptr column_matcher_); /// Get except transformer type ExceptColumnTransformerType getExceptTransformerType() const @@ -229,9 +224,11 @@ protected: private: ExceptColumnTransformerType except_transformer_type; - bool is_strict; Names except_column_names; std::shared_ptr column_matcher; + bool is_strict = false; + + static constexpr size_t children_size = 0; }; class ReplaceColumnTransformerNode; @@ -263,12 +260,6 @@ public: return ColumnTransfomerType::REPLACE; } - /// Is replace column transformer strict - bool isStrict() const - { - return is_strict; - } - /// Get replacements ListNode & getReplacements() const { @@ -287,6 +278,12 @@ public: return replacements_names; } + /// Is replace column transformer strict + bool isStrict() const + { + return is_strict; + } + /** Returns replacement expression if for expression name replacements exists, nullptr otherwise. * Returned replacement expression must be cloned by caller. */ @@ -304,11 +301,11 @@ protected: QueryTreeNodePtr cloneImpl() const override; private: - ReplaceColumnTransformerNode() = default; - - bool is_strict; Names replacements_names; + bool is_strict = false; + static constexpr size_t replacements_child_index = 0; + static constexpr size_t children_size = replacements_child_index + 1; }; } diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 9539f9fa142..4a292d5501b 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -17,7 +17,8 @@ namespace DB { ConstantNode::ConstantNode(ConstantValuePtr constant_value_) - : constant_value(std::move(constant_value_)) + : IQueryTreeNode(children_size) + , constant_value(std::move(constant_value_)) , value_string(applyVisitor(FieldVisitorToString(), constant_value->getValue())) { } diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index 5730dba9218..25931e848d5 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -75,6 +75,8 @@ protected: private: ConstantValuePtr constant_value; String value_string; + + static constexpr size_t children_size = 0; }; } diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index a5a20761556..feea942d5a2 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -18,9 +18,9 @@ namespace DB { FunctionNode::FunctionNode(String function_name_) - : function_name(function_name_) + : IQueryTreeNode(children_size) + , function_name(function_name_) { - children.resize(children_size); children[parameters_child_index] = std::make_shared(); children[arguments_child_index] = std::make_shared(); } @@ -201,7 +201,9 @@ QueryTreeNodePtr FunctionNode::cloneImpl() const { auto result_function = std::make_shared(function_name); - /// This is valid for clone method function or aggregate function must be stateless + /** This is valid for clone method to reuse same function pointers + * because ordinary functions or aggregate functions must be stateless. + */ result_function->function = function; result_function->aggregate_function = aggregate_function; result_function->result_type = result_type; diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index 7f97aa89178..d8dd869c25e 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -162,7 +162,7 @@ public: } /** Resolve function node as non aggregate function. - * It is important that function name is update with resolved function name. + * It is important that function name is updated with resolved function name. * Main motivation for this is query tree optimizations. * Assume we have `multiIf` function with single argument, it can be converted to `if` function. * Function name must be updated accordingly. @@ -170,13 +170,13 @@ public: void resolveAsFunction(FunctionOverloadResolverPtr function_value, DataTypePtr result_type_value); /** Resolve function node as aggregate function. - * It is important that function name is update with resolved function name. + * It is important that function name is updated with resolved function name. * Main motivation for this is query tree optimizations. */ void resolveAsAggregateFunction(AggregateFunctionPtr aggregate_function_value, DataTypePtr result_type_value); /** Resolve function node as window function. - * It is important that function name is update with resolved function name. + * It is important that function name is updated with resolved function name. * Main motivation for this is query tree optimizations. */ void resolveAsWindowFunction(AggregateFunctionPtr window_function_value, DataTypePtr result_type_value); @@ -216,16 +216,16 @@ protected: QueryTreeNodePtr cloneImpl() const override; private: - static constexpr size_t parameters_child_index = 0; - static constexpr size_t arguments_child_index = 1; - static constexpr size_t window_child_index = 2; - static constexpr size_t children_size = window_child_index + 1; - String function_name; FunctionOverloadResolverPtr function; AggregateFunctionPtr aggregate_function; DataTypePtr result_type; ConstantValuePtr constant_value; + + static constexpr size_t parameters_child_index = 0; + static constexpr size_t arguments_child_index = 1; + static constexpr size_t window_child_index = 2; + static constexpr size_t children_size = window_child_index + 1; }; } diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 2f555b29708..f6354c7f9a6 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -42,6 +42,11 @@ const char * toString(QueryTreeNodeType type) } } +IQueryTreeNode::IQueryTreeNode(size_t children_size) +{ + children.resize(children_size); +} + String IQueryTreeNode::dumpTree() const { WriteBufferFromOwnString buff; diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 22b3b658f3e..f462de38f2e 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -252,6 +252,8 @@ public: virtual void updateTreeHashImpl(HashState & hash_state) const = 0; protected: + /// Construct query tree node and resize children to children size + explicit IQueryTreeNode(size_t children_size); /** Subclass node must convert itself to AST. * Subclass must convert children to AST. diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp index f9128fa87c9..172a3ef86be 100644 --- a/src/Analyzer/IdentifierNode.cpp +++ b/src/Analyzer/IdentifierNode.cpp @@ -10,6 +10,17 @@ namespace DB { +IdentifierNode::IdentifierNode(Identifier identifier_) + : IQueryTreeNode(children_size) + , identifier(std::move(identifier_)) +{} + +IdentifierNode::IdentifierNode(Identifier identifier_, TableExpressionModifiers table_expression_modifiers_) + : IQueryTreeNode(children_size) + , identifier(std::move(identifier_)) + , table_expression_modifiers(std::move(table_expression_modifiers_)) +{} + void IdentifierNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "IDENTIFIER id: " << format_state.getNodeId(this); diff --git a/src/Analyzer/IdentifierNode.h b/src/Analyzer/IdentifierNode.h index a47fb283b15..326dc43c030 100644 --- a/src/Analyzer/IdentifierNode.h +++ b/src/Analyzer/IdentifierNode.h @@ -18,19 +18,14 @@ class IdentifierNode final : public IQueryTreeNode { public: /// Construct identifier node with identifier - explicit IdentifierNode(Identifier identifier_) - : identifier(std::move(identifier_)) - {} + explicit IdentifierNode(Identifier identifier_); /** Construct identifier node with identifier and table expression modifiers * when identifier node is part of JOIN TREE. * * Example: SELECT * FROM test_table SAMPLE 0.1 OFFSET 0.1 FINAL */ - explicit IdentifierNode(Identifier identifier_, TableExpressionModifiers table_expression_modifiers_) - : identifier(std::move(identifier_)) - , table_expression_modifiers(std::move(table_expression_modifiers_)) - {} + explicit IdentifierNode(Identifier identifier_, TableExpressionModifiers table_expression_modifiers_); /// Get identifier const Identifier & getIdentifier() const @@ -45,7 +40,7 @@ public: } /// Get table expression modifiers - std::optional getTableExpressionModifiers() const + const std::optional & getTableExpressionModifiers() const { return table_expression_modifiers; } @@ -74,6 +69,8 @@ protected: private: Identifier identifier; std::optional table_expression_modifiers; + + static constexpr size_t children_size = 0; }; } diff --git a/src/Analyzer/InterpolateNode.cpp b/src/Analyzer/InterpolateNode.cpp index 628caaaf219..3bb211343fa 100644 --- a/src/Analyzer/InterpolateNode.cpp +++ b/src/Analyzer/InterpolateNode.cpp @@ -11,8 +11,8 @@ namespace DB { InterpolateNode::InterpolateNode(QueryTreeNodePtr expression_, QueryTreeNodePtr interpolate_expression_) + : IQueryTreeNode(children_size) { - children.resize(children_size); children[expression_child_index] = std::move(expression_); children[interpolate_expression_child_index] = std::move(interpolate_expression_); } diff --git a/src/Analyzer/InterpolateNode.h b/src/Analyzer/InterpolateNode.h index d9efffdc79f..55270594d2f 100644 --- a/src/Analyzer/InterpolateNode.h +++ b/src/Analyzer/InterpolateNode.h @@ -20,25 +20,25 @@ public: /// Initialize interpolate node with expression and interpolate expression explicit InterpolateNode(QueryTreeNodePtr expression_, QueryTreeNodePtr interpolate_expression_); - /// Get expression + /// Get expression to interpolate const QueryTreeNodePtr & getExpression() const { return children[expression_child_index]; } - /// Get expression + /// Get expression to interpolate QueryTreeNodePtr & getExpression() { return children[expression_child_index]; } - /// Get expression + /// Get interpolate expression const QueryTreeNodePtr & getInterpolateExpression() const { return children[interpolate_expression_child_index]; } - /// Get expression + /// Get interpolate expression QueryTreeNodePtr & getInterpolateExpression() { return children[interpolate_expression_child_index]; diff --git a/src/Analyzer/JoinNode.cpp b/src/Analyzer/JoinNode.cpp index cd006efdfd3..08213b5622d 100644 --- a/src/Analyzer/JoinNode.cpp +++ b/src/Analyzer/JoinNode.cpp @@ -21,11 +21,11 @@ JoinNode::JoinNode(QueryTreeNodePtr left_table_expression_, JoinLocality locality_, JoinStrictness strictness_, JoinKind kind_) - : locality(locality_) + : IQueryTreeNode(children_size) + , locality(locality_) , strictness(strictness_) , kind(kind_) { - children.resize(children_size); children[left_table_expression_child_index] = std::move(left_table_expression_); children[right_table_expression_child_index] = std::move(right_table_expression_); children[join_expression_child_index] = std::move(join_expression_); @@ -37,7 +37,8 @@ JoinNode::JoinNode(QueryTreeNodePtr left_table_expression_, JoinLocality locality_, JoinStrictness strictness_, JoinKind kind_) - : locality(locality_) + : IQueryTreeNode(children_size) + , locality(locality_) , strictness(strictness_) , kind(kind_) { @@ -47,14 +48,6 @@ JoinNode::JoinNode(QueryTreeNodePtr left_table_expression_, children[join_expression_child_index] = std::make_shared(std::move(using_identifiers)); } -JoinNode::JoinNode(JoinLocality locality_, - JoinStrictness strictness_, - JoinKind kind_) - : locality(locality_) - , strictness(strictness_) - , kind(kind_) -{} - void JoinNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "JOIN id: " << format_state.getNodeId(this); @@ -129,11 +122,12 @@ ASTPtr JoinNode::toASTImpl() const if (children[join_expression_child_index]) { auto join_expression_ast = children[join_expression_child_index]->toAST(); + join_ast->children.push_back(std::move(join_expression_ast)); if (children[join_expression_child_index]->getNodeType() == QueryTreeNodeType::LIST) - join_ast->using_expression_list = std::move(join_expression_ast); + join_ast->using_expression_list = join_ast->children.back(); else - join_ast->on_expression = std::move(join_expression_ast); + join_ast->on_expression = join_ast->children.back(); } addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[right_table_expression_child_index]); @@ -147,8 +141,7 @@ ASTPtr JoinNode::toASTImpl() const QueryTreeNodePtr JoinNode::cloneImpl() const { - JoinNodePtr result_join_node(new JoinNode(locality, strictness, kind)); - return result_join_node; + return std::make_shared(getLeftTableExpression(), getRightTableExpression(), getJoinExpression(), locality, strictness, kind); } } diff --git a/src/Analyzer/JoinNode.h b/src/Analyzer/JoinNode.h index 3bde9a25b84..738fdccd6db 100644 --- a/src/Analyzer/JoinNode.h +++ b/src/Analyzer/JoinNode.h @@ -14,8 +14,17 @@ namespace DB { -/** Array join node represents array join in query tree. - * Example: SELECT id FROM test_table ARRAY JOIN [1, 2, 3]. +/** Join node represents join in query tree. + * + * For JOIN without join expression, JOIN expression is null. + * Example: SELECT id FROM test_table_1 AS t1, test_table_2 AS t2; + * + * For JOIN with USING, JOIN expression contains list of identifier nodes. These nodes must be resolved + * during query analysis pass. + * Example: SELECT id FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 USING (id); + * + * For JOIN with ON, JOIN expression contains single expression. + * Example: SELECT id FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t1.id; */ class JoinNode; using JoinNodePtr = std::shared_ptr; @@ -130,10 +139,6 @@ protected: QueryTreeNodePtr cloneImpl() const override; private: - JoinNode(JoinLocality locality_, - JoinStrictness strictness_, - JoinKind kind_); - JoinLocality locality = JoinLocality::Unspecified; JoinStrictness strictness = JoinStrictness::Unspecified; JoinKind kind = JoinKind::Inner; diff --git a/src/Analyzer/LambdaNode.cpp b/src/Analyzer/LambdaNode.cpp index 8f6d47e584d..8a40072c103 100644 --- a/src/Analyzer/LambdaNode.cpp +++ b/src/Analyzer/LambdaNode.cpp @@ -10,10 +10,9 @@ namespace DB { LambdaNode::LambdaNode(Names argument_names_, QueryTreeNodePtr expression_) - : argument_names(std::move(argument_names_)) + : IQueryTreeNode(children_size) + , argument_names(std::move(argument_names_)) { - children.resize(2); - auto arguments_list_node = std::make_shared(); auto & nodes = arguments_list_node->getNodes(); @@ -88,11 +87,7 @@ ASTPtr LambdaNode::toASTImpl() const QueryTreeNodePtr LambdaNode::cloneImpl() const { - LambdaNodePtr result_lambda(new LambdaNode()); - - result_lambda->argument_names = argument_names; - - return result_lambda; + return std::make_shared(argument_names, getExpression()); } } diff --git a/src/Analyzer/LambdaNode.h b/src/Analyzer/LambdaNode.h index bbd2035acc5..3d4ff1c6c55 100644 --- a/src/Analyzer/LambdaNode.h +++ b/src/Analyzer/LambdaNode.h @@ -16,7 +16,7 @@ namespace DB * * Initially lambda is initialized with argument names and expression query tree node. * During query analysis if expression is not resolved lambda must be resolved. - * Lambda is resolved if lambda expression is resolved. + * Lambda is in resolved state if lambda body expression is in resolved state. * * It is important that lambda expression result type can depend on arguments types. * Example: WITH (x -> x) as lambda SELECT lambda(1), lambda('string_value'). @@ -112,12 +112,11 @@ protected: QueryTreeNodePtr cloneImpl() const override; private: - LambdaNode() = default; + Names argument_names; static constexpr size_t arguments_child_index = 0; static constexpr size_t expression_child_index = 1; - - Names argument_names; + static constexpr size_t children_size = expression_child_index + 1; }; } diff --git a/src/Analyzer/ListNode.cpp b/src/Analyzer/ListNode.cpp index 59b42d384de..f9affa133cf 100644 --- a/src/Analyzer/ListNode.cpp +++ b/src/Analyzer/ListNode.cpp @@ -11,7 +11,12 @@ namespace DB { +ListNode::ListNode() + : IQueryTreeNode(0 /*children_size*/) +{} + ListNode::ListNode(QueryTreeNodes nodes) + : IQueryTreeNode(0 /*children_size*/) { children = std::move(nodes); } diff --git a/src/Analyzer/ListNode.h b/src/Analyzer/ListNode.h index f93043144e0..0f44b500057 100644 --- a/src/Analyzer/ListNode.h +++ b/src/Analyzer/ListNode.h @@ -17,7 +17,7 @@ class ListNode final : public IQueryTreeNode { public: /// Initialize list node with empty nodes - ListNode() = default; + ListNode(); /// Initialize list node with nodes explicit ListNode(QueryTreeNodes nodes); diff --git a/src/Analyzer/MatcherNode.cpp b/src/Analyzer/MatcherNode.cpp index 6393193ec01..21a80b1608b 100644 --- a/src/Analyzer/MatcherNode.cpp +++ b/src/Analyzer/MatcherNode.cpp @@ -86,7 +86,8 @@ MatcherNode::MatcherNode(MatcherNodeType matcher_type_, Identifiers columns_identifiers_, std::shared_ptr columns_matcher_, ColumnTransformersNodes column_transformers_) - : matcher_type(matcher_type_) + : IQueryTreeNode(children_size) + , matcher_type(matcher_type_) , qualified_identifier(qualified_identifier_) , columns_identifiers(columns_identifiers_) , columns_matcher(columns_matcher_) @@ -99,7 +100,6 @@ MatcherNode::MatcherNode(MatcherNodeType matcher_type_, for (auto && column_transformer : column_transformers_) column_transformers_nodes.emplace_back(std::move(column_transformer)); - children.resize(1); children[column_transformers_child_index] = std::move(column_transformers_list_node); columns_identifiers_set.reserve(columns_identifiers.size()); @@ -155,30 +155,49 @@ void MatcherNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, String MatcherNode::getName() const { - if (matcher_type == MatcherNodeType::ASTERISK) - return "*"; - WriteBufferFromOwnString buffer; - buffer << "COLUMNS("; - if (columns_matcher) + if (!qualified_identifier.empty()) + buffer << qualified_identifier.getFullName() << '.'; + + if (matcher_type == MatcherNodeType::ASTERISK) { - buffer << ' ' << columns_matcher->pattern(); + buffer << '*'; } - else if (matcher_type == MatcherNodeType::COLUMNS_LIST) + else { - size_t columns_identifiers_size = columns_identifiers.size(); - for (size_t i = 0; i < columns_identifiers_size; ++i) - { - buffer << columns_identifiers[i].getFullName(); + buffer << "COLUMNS("; - if (i + 1 != columns_identifiers_size) - buffer << ", "; + if (columns_matcher) + { + buffer << ' ' << columns_matcher->pattern(); + } + else if (matcher_type == MatcherNodeType::COLUMNS_LIST) + { + size_t columns_identifiers_size = columns_identifiers.size(); + for (size_t i = 0; i < columns_identifiers_size; ++i) + { + buffer << columns_identifiers[i].getFullName(); + + if (i + 1 != columns_identifiers_size) + buffer << ", "; + } } } buffer << ')'; - /// TODO: Transformers + + const auto & column_transformers = getColumnTransformers().getNodes(); + size_t column_transformers_size = column_transformers.size(); + + for (size_t i = 0; i < column_transformers_size; ++i) + { + const auto & column_transformer = column_transformers[i]; + buffer << column_transformer->getName(); + + if (i + 1 != column_transformers_size) + buffer << ' '; + } return buffer.str(); } diff --git a/src/Analyzer/MatcherNode.h b/src/Analyzer/MatcherNode.h index ebbde9cd4fd..b489f98588a 100644 --- a/src/Analyzer/MatcherNode.h +++ b/src/Analyzer/MatcherNode.h @@ -174,6 +174,7 @@ private: std::unordered_set columns_identifiers_set; static constexpr size_t column_transformers_child_index = 0; + static constexpr size_t children_size = column_transformers_child_index + 1; }; } diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index c144cad7b83..6acc70d898b 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -23,8 +23,8 @@ namespace DB { QueryNode::QueryNode() + : IQueryTreeNode(children_size) { - children.resize(children_size); children[with_child_index] = std::make_shared(); children[projection_child_index] = std::make_shared(); children[group_by_child_index] = std::make_shared(); diff --git a/src/Analyzer/SortNode.cpp b/src/Analyzer/SortNode.cpp index f9dd2054134..5bdb9b9adf7 100644 --- a/src/Analyzer/SortNode.cpp +++ b/src/Analyzer/SortNode.cpp @@ -26,12 +26,12 @@ SortNode::SortNode(QueryTreeNodePtr expression_, std::optional nulls_sort_direction_, std::shared_ptr collator_, bool with_fill_) - : sort_direction(sort_direction_) + : IQueryTreeNode(children_size) + , sort_direction(sort_direction_) , nulls_sort_direction(nulls_sort_direction_) , collator(std::move(collator_)) , with_fill(with_fill_) { - children.resize(children_size); children[sort_expression_child_index] = std::move(expression_); } diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index 0d18d5a3d00..b0ade7ca1c4 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -19,10 +19,10 @@ namespace ErrorCodes } TableFunctionNode::TableFunctionNode(String table_function_name_) - : table_function_name(table_function_name_) + : IQueryTreeNode(children_size) + , table_function_name(table_function_name_) , storage_id("system", "one") { - children.resize(1); children[arguments_child_index] = std::make_shared(); } diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index 6f2659bd8de..979a930542b 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -141,14 +141,15 @@ protected: QueryTreeNodePtr cloneImpl() const override; private: - static constexpr size_t arguments_child_index = 0; - String table_function_name; TableFunctionPtr table_function; StoragePtr storage; StorageID storage_id; StorageSnapshotPtr storage_snapshot; std::optional table_expression_modifiers; + + static constexpr size_t arguments_child_index = 0; + static constexpr size_t children_size = arguments_child_index + 1; }; } diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index e128987d179..fc25d192f3a 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -13,16 +13,19 @@ namespace DB { -TableNode::TableNode(StoragePtr storage_, TableLockHolder storage_lock_, StorageSnapshotPtr storage_snapshot_) - : storage(std::move(storage_)) - , storage_id(storage->getStorageID()) +TableNode::TableNode(StoragePtr storage_, StorageID storage_id_, TableLockHolder storage_lock_, StorageSnapshotPtr storage_snapshot_) + : IQueryTreeNode(children_size) + , storage(std::move(storage_)) + , storage_id(std::move(storage_id_)) , storage_lock(std::move(storage_lock_)) , storage_snapshot(std::move(storage_snapshot_)) +{} + +TableNode::TableNode(StoragePtr storage_, TableLockHolder storage_lock_, StorageSnapshotPtr storage_snapshot_) + : TableNode(storage_, storage_->getStorageID(), std::move(storage_lock_), std::move(storage_snapshot_)) { } -TableNode::TableNode() : storage_id("system", "one") {} - void TableNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "TABLE id: " << format_state.getNodeId(this); @@ -75,12 +78,7 @@ ASTPtr TableNode::toASTImpl() const QueryTreeNodePtr TableNode::cloneImpl() const { - TableNodePtr result_table_node(new TableNode()); - - result_table_node->storage = storage; - result_table_node->storage_id = storage_id; - result_table_node->storage_lock = storage_lock; - result_table_node->storage_snapshot = storage_snapshot; + auto result_table_node = std::make_shared(storage, storage_id, storage_lock, storage_snapshot); result_table_node->table_expression_modifiers = table_expression_modifiers; return result_table_node; diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index b06166c5b0d..216dc516246 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -27,6 +27,9 @@ using TableNodePtr = std::shared_ptr; class TableNode : public IQueryTreeNode { public: + /// Construct table node with storage, storage id, storage lock, storage snapshot + explicit TableNode(StoragePtr storage_, StorageID storage_id_, TableLockHolder storage_lock_, StorageSnapshotPtr storage_snapshot_); + /// Construct table node with storage, storage lock, storage snapshot explicit TableNode(StoragePtr storage_, TableLockHolder storage_lock_, StorageSnapshotPtr storage_snapshot_); @@ -54,14 +57,6 @@ public: return storage_lock; } - /** Move table lock out of table node. - * After using this method table node state becomes invalid. - */ - TableLockHolder && moveStorageLock() - { - return std::move(storage_lock); - } - /// Return true if table node has table expression modifiers, false otherwise bool hasTableExpressionModifiers() const { @@ -99,13 +94,13 @@ protected: QueryTreeNodePtr cloneImpl() const override; private: - TableNode(); - StoragePtr storage; StorageID storage_id; TableLockHolder storage_lock; StorageSnapshotPtr storage_snapshot; std::optional table_expression_modifiers; + + static constexpr size_t children_size = 0; }; } diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 4e67fa5b169..b3fa07a828f 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -33,8 +33,8 @@ namespace ErrorCodes } UnionNode::UnionNode() + : IQueryTreeNode(children_size) { - children.resize(children_size); children[queries_child_index] = std::make_shared(); } diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 394e3ae0471..9640694933b 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -47,10 +47,22 @@ bool isNodePartOfTree(const IQueryTreeNode * node, const IQueryTreeNode * root) bool isNameOfInFunction(const std::string & function_name) { - bool is_special_function_in = function_name == "in" || function_name == "globalIn" || function_name == "notIn" || function_name == "globalNotIn" || - function_name == "nullIn" || function_name == "globalNullIn" || function_name == "notNullIn" || function_name == "globalNotNullIn" || - function_name == "inIgnoreSet" || function_name == "globalInIgnoreSet" || function_name == "notInIgnoreSet" || function_name == "globalNotInIgnoreSet" || - function_name == "nullInIgnoreSet" || function_name == "globalNullInIgnoreSet" || function_name == "notNullInIgnoreSet" || function_name == "globalNotNullInIgnoreSet"; + bool is_special_function_in = function_name == "in" || + function_name == "globalIn" || + function_name == "notIn" || + function_name == "globalNotIn" || + function_name == "nullIn" || + function_name == "globalNullIn" || + function_name == "notNullIn" || + function_name == "globalNotNullIn" || + function_name == "inIgnoreSet" || + function_name == "globalInIgnoreSet" || + function_name == "notInIgnoreSet" || + function_name == "globalNotInIgnoreSet" || + function_name == "nullInIgnoreSet" || + function_name == "globalNullInIgnoreSet" || + function_name == "notNullInIgnoreSet" || + function_name == "globalNotNullInIgnoreSet"; return is_special_function_in; } diff --git a/src/Analyzer/WindowNode.cpp b/src/Analyzer/WindowNode.cpp index 94fa8f83505..0378c8746ed 100644 --- a/src/Analyzer/WindowNode.cpp +++ b/src/Analyzer/WindowNode.cpp @@ -11,9 +11,9 @@ namespace DB { WindowNode::WindowNode(WindowFrame window_frame_) - : window_frame(std::move(window_frame_)) + : IQueryTreeNode(children_size) + , window_frame(std::move(window_frame_)) { - children.resize(children_size); children[partition_by_child_index] = std::make_shared(); children[order_by_child_index] = std::make_shared(); } diff --git a/src/Analyzer/tests/gtest_query_tree_node.cpp b/src/Analyzer/tests/gtest_query_tree_node.cpp index 0940aa5861d..f29a0f115d7 100644 --- a/src/Analyzer/tests/gtest_query_tree_node.cpp +++ b/src/Analyzer/tests/gtest_query_tree_node.cpp @@ -11,6 +11,8 @@ using namespace DB; class SourceNode final : public IQueryTreeNode { public: + SourceNode() : IQueryTreeNode(0 /*children_size*/) {} + QueryTreeNodeType getNodeType() const override { return QueryTreeNodeType::TABLE; From 2f282eeec91644697b47ccfa423ffc927db2cf7c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 7 Oct 2022 15:20:45 +0200 Subject: [PATCH 127/188] Updated InDepthQueryTreeVisitor implementation --- .../CollectAggregateFunctionNodes.cpp | 51 +++--- src/Analyzer/CollectWindowFunctionNodes.cpp | 49 +++-- src/Analyzer/CountDistinctPass.cpp | 18 +- src/Analyzer/CustomizeFunctionsPass.cpp | 52 +++--- src/Analyzer/IfChainToMultiIfPass.cpp | 24 +-- src/Analyzer/IfConstantConditionPass.cpp | 18 +- src/Analyzer/InDepthQueryTreeVisitor.h | 72 +++++--- src/Analyzer/MultiIfToIfPass.cpp | 24 +-- src/Analyzer/NormalizeCountVariantsPass.cpp | 18 +- src/Analyzer/OrderByTupleEliminationPass.cpp | 18 +- src/Analyzer/QueryAnalysisPass.cpp | 168 ++++++++---------- src/Analyzer/QueryTreeBuilder.cpp | 1 - .../InterpreterSelectQueryAnalyzer.cpp | 44 +---- src/Planner/CollectSets.cpp | 22 +-- src/Planner/CollectTableExpressionData.cpp | 27 ++- src/Planner/Planner.cpp | 1 - src/Planner/PlannerAggregation.cpp | 81 ++++----- src/Planner/PlannerWindowFunctions.cpp | 1 - 18 files changed, 275 insertions(+), 414 deletions(-) diff --git a/src/Analyzer/CollectAggregateFunctionNodes.cpp b/src/Analyzer/CollectAggregateFunctionNodes.cpp index 5a60d63ca8b..8629fa09902 100644 --- a/src/Analyzer/CollectAggregateFunctionNodes.cpp +++ b/src/Analyzer/CollectAggregateFunctionNodes.cpp @@ -14,71 +14,64 @@ namespace ErrorCodes namespace { -class CollectAggregateFunctionNodesMatcher +class CollectAggregateFunctionNodesVisitor : public ConstInDepthQueryTreeVisitor { public: - using Visitor = ConstInDepthQueryTreeVisitor; + explicit CollectAggregateFunctionNodesVisitor(QueryTreeNodes * aggregate_function_nodes_) + : aggregate_function_nodes(aggregate_function_nodes_) + {} - struct Data - { - Data() = default; + explicit CollectAggregateFunctionNodesVisitor(String assert_no_aggregates_place_message_) + : assert_no_aggregates_place_message(std::move(assert_no_aggregates_place_message_)) + {} - String assert_no_aggregates_place_message; - QueryTreeNodes * aggregate_function_nodes = nullptr; - }; - - static void visit(const QueryTreeNodePtr & node, Data & data) + void visitImpl(const QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || !function_node->isAggregateFunction()) return; - if (!data.assert_no_aggregates_place_message.empty()) + if (!assert_no_aggregates_place_message.empty()) throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, "Aggregate function {} is found {} in query", function_node->getName(), - data.assert_no_aggregates_place_message); + assert_no_aggregates_place_message); - if (data.aggregate_function_nodes) - data.aggregate_function_nodes->push_back(node); + if (aggregate_function_nodes) + aggregate_function_nodes->push_back(node); } static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); } -}; -using CollectAggregateFunctionNodesVisitor = CollectAggregateFunctionNodesMatcher::Visitor; +private: + String assert_no_aggregates_place_message; + QueryTreeNodes * aggregate_function_nodes = nullptr; +}; } QueryTreeNodes collectAggregateFunctionNodes(const QueryTreeNodePtr & node) { QueryTreeNodes result; - - CollectAggregateFunctionNodesVisitor::Data data; - data.aggregate_function_nodes = &result; - - CollectAggregateFunctionNodesVisitor(data).visit(node); + CollectAggregateFunctionNodesVisitor visitor(&result); + visitor.visit(node); return result; } void collectAggregateFunctionNodes(const QueryTreeNodePtr & node, QueryTreeNodes & result) { - CollectAggregateFunctionNodesVisitor::Data data; - data.aggregate_function_nodes = &result; - - CollectAggregateFunctionNodesVisitor(data).visit(node); + CollectAggregateFunctionNodesVisitor visitor(&result); + visitor.visit(node); } void assertNoAggregateFunctionNodes(const QueryTreeNodePtr & node, const String & assert_no_aggregates_place_message) { - CollectAggregateFunctionNodesVisitor::Data data; - data.assert_no_aggregates_place_message = assert_no_aggregates_place_message; - - CollectAggregateFunctionNodesVisitor(data).visit(node); + CollectAggregateFunctionNodesVisitor visitor(assert_no_aggregates_place_message); + visitor.visit(node); } } diff --git a/src/Analyzer/CollectWindowFunctionNodes.cpp b/src/Analyzer/CollectWindowFunctionNodes.cpp index ba3e0bde909..89c2e8ee626 100644 --- a/src/Analyzer/CollectWindowFunctionNodes.cpp +++ b/src/Analyzer/CollectWindowFunctionNodes.cpp @@ -15,53 +15,49 @@ namespace ErrorCodes namespace { -class CollecWindowFunctionNodesMatcher +class CollectWindowFunctionNodeVisitor : public ConstInDepthQueryTreeVisitor { public: - using Visitor = ConstInDepthQueryTreeVisitor; + explicit CollectWindowFunctionNodeVisitor(QueryTreeNodes * window_function_nodes_) + : window_function_nodes(window_function_nodes_) + {} - struct Data - { - Data() = default; + explicit CollectWindowFunctionNodeVisitor(String assert_no_window_functions_place_message_) + : assert_no_window_functions_place_message(std::move(assert_no_window_functions_place_message_)) + {} - String assert_no_window_functions_place_message; - QueryTreeNodes * window_function_nodes = nullptr; - }; - - static void visit(const QueryTreeNodePtr & node, Data & data) + void visitImpl(const QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || !function_node->isWindowFunction()) return; - if (!data.assert_no_window_functions_place_message.empty()) + if (!assert_no_window_functions_place_message.empty()) throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, "Window function {} is found {} in query", function_node->getName(), - data.assert_no_window_functions_place_message); + assert_no_window_functions_place_message); - if (data.window_function_nodes) - data.window_function_nodes->push_back(node); + if (window_function_nodes) + window_function_nodes->push_back(node); } static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); } -}; -using CollectWindowFunctionNodesVisitor = CollecWindowFunctionNodesMatcher::Visitor; +private: + QueryTreeNodes * window_function_nodes = nullptr; + String assert_no_window_functions_place_message; +}; } QueryTreeNodes collectWindowFunctionNodes(const QueryTreeNodePtr & node) { QueryTreeNodes window_function_nodes; - - CollectWindowFunctionNodesVisitor::Data data; - data.window_function_nodes = &window_function_nodes; - - CollectWindowFunctionNodesVisitor visitor(data); + CollectWindowFunctionNodeVisitor visitor(&window_function_nodes); visitor.visit(node); return window_function_nodes; @@ -69,19 +65,14 @@ QueryTreeNodes collectWindowFunctionNodes(const QueryTreeNodePtr & node) void collectWindowFunctionNodes(const QueryTreeNodePtr & node, QueryTreeNodes & result) { - CollectWindowFunctionNodesVisitor::Data data; - data.window_function_nodes = &result; - - CollectWindowFunctionNodesVisitor visitor(data); + CollectWindowFunctionNodeVisitor visitor(&result); visitor.visit(node); } void assertNoWindowFunctionNodes(const QueryTreeNodePtr & node, const String & assert_no_window_functions_place_message) { - CollectWindowFunctionNodesVisitor::Data data; - data.assert_no_window_functions_place_message = assert_no_window_functions_place_message; - - CollectWindowFunctionNodesVisitor(data).visit(node); + CollectWindowFunctionNodeVisitor visitor(assert_no_window_functions_place_message); + visitor.visit(node); } } diff --git a/src/Analyzer/CountDistinctPass.cpp b/src/Analyzer/CountDistinctPass.cpp index 69d0d7f5366..35ceff30337 100644 --- a/src/Analyzer/CountDistinctPass.cpp +++ b/src/Analyzer/CountDistinctPass.cpp @@ -14,16 +14,10 @@ namespace DB namespace { -class CountDistinctMatcher +class CountDistinctVisitor : public InDepthQueryTreeVisitor { public: - using Visitor = InDepthQueryTreeVisitor; - - struct Data - { - }; - - static void visit(QueryTreeNodePtr & node, Data &) + static void visitImpl(QueryTreeNodePtr & node) { auto * query_node = node->as(); @@ -77,19 +71,13 @@ public: function_node->resolveAsAggregateFunction(std::move(aggregate_function), std::move(result_type)); function_node->getArguments().getNodes().clear(); } - - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) - { - return true; - } }; } void CountDistinctPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) { - CountDistinctMatcher::Data data{}; - CountDistinctMatcher::Visitor visitor(data); + CountDistinctVisitor visitor; visitor.visit(query_tree_node); } diff --git a/src/Analyzer/CustomizeFunctionsPass.cpp b/src/Analyzer/CustomizeFunctionsPass.cpp index 6a21656fb3e..298c8afb516 100644 --- a/src/Analyzer/CustomizeFunctionsPass.cpp +++ b/src/Analyzer/CustomizeFunctionsPass.cpp @@ -15,36 +15,33 @@ namespace DB namespace { -class CustomizeFunctionsMatcher +class CustomizeFunctionsVisitor : public InDepthQueryTreeVisitor { public: - using Visitor = InDepthQueryTreeVisitor; + explicit CustomizeFunctionsVisitor(ContextPtr & context_) + : context(context_) + {} - struct Data - { - ContextPtr & context; - }; - - static void visit(QueryTreeNodePtr & node, Data & data) + void visitImpl(QueryTreeNodePtr & node) const { auto * function_node = node->as(); if (!function_node) return; - const auto & settings = data.context->getSettingsRef(); + const auto & settings = context->getSettingsRef(); /// After successful function replacement function name and function name lowercase must be recalculated auto function_name = function_node->getFunctionName(); auto function_name_lowercase = Poco::toLower(function_name); - if (function_node->isAggregateFunction()) + if (function_node->isAggregateFunction() || function_node->isWindowFunction()) { auto count_distinct_implementation_function_name = String(settings.count_distinct_implementation); /// Replace countDistinct with countDistinct implementation if (function_name_lowercase == "countdistinct") { - resolveFunctionNodeAsAggregateFunction(*function_node, count_distinct_implementation_function_name); + resolveAggregateOrWindowFunctionNode(*function_node, count_distinct_implementation_function_name); function_name = function_node->getFunctionName(); function_name_lowercase = Poco::toLower(function_name); } @@ -52,7 +49,7 @@ public: /// Replace countDistinct with countIfDistinct with countDistinctIf implementation if (function_name_lowercase == "countifdistinct") { - resolveFunctionNodeAsAggregateFunction(*function_node, count_distinct_implementation_function_name + "If"); + resolveAggregateOrWindowFunctionNode(*function_node, count_distinct_implementation_function_name + "If"); function_name = function_node->getFunctionName(); function_name_lowercase = Poco::toLower(function_name); } @@ -62,7 +59,7 @@ public: { size_t prefix_length = function_name_lowercase.size() - strlen("ifdistinct"); auto updated_function_name = function_name_lowercase.substr(0, prefix_length) + "DistinctIf"; - resolveFunctionNodeAsAggregateFunction(*function_node, updated_function_name); + resolveAggregateOrWindowFunctionNode(*function_node, updated_function_name); function_name = function_node->getFunctionName(); function_name_lowercase = Poco::toLower(function_name); } @@ -74,7 +71,7 @@ public: if (function_properies && !function_properies->returns_default_when_only_null) { auto updated_function_name = function_name + "OrNull"; - resolveFunctionNodeAsAggregateFunction(*function_node, updated_function_name); + resolveAggregateOrWindowFunctionNode(*function_node, updated_function_name); function_name = function_node->getFunctionName(); function_name_lowercase = Poco::toLower(function_name); } @@ -99,7 +96,7 @@ public: continue; auto updated_function_name = function_name_lowercase.substr(0, function_name_size - suffix.size()) + "OrNull" + String(suffix); - resolveFunctionNodeAsAggregateFunction(*function_node, updated_function_name); + resolveAggregateOrWindowFunctionNode(*function_node, updated_function_name); function_name = function_node->getFunctionName(); function_name_lowercase = Poco::toLower(function_name); break; @@ -126,7 +123,7 @@ public: { if (function_name_lowercase == in_function_name) { - resolveFunctionNodeAsFunction(*function_node, String(in_function_name_to_replace), data); + resolveOrdinaryFunctionNode(*function_node, String(in_function_name_to_replace)); function_name = function_node->getFunctionName(); function_name_lowercase = Poco::toLower(function_name); break; @@ -135,12 +132,7 @@ public: } } - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) - { - return true; - } - - static inline void resolveFunctionNodeAsAggregateFunction(FunctionNode & function_node, const String & aggregate_function_name) + static inline void resolveAggregateOrWindowFunctionNode(FunctionNode & function_node, const String & aggregate_function_name) { auto function_result_type = function_node.getResultType(); auto function_aggregate_function = function_node.getAggregateFunction(); @@ -150,23 +142,29 @@ public: function_aggregate_function->getArgumentTypes(), function_aggregate_function->getParameters(), properties); - function_node.resolveAsAggregateFunction(std::move(aggregate_function), std::move(function_result_type)); + + if (function_node.isAggregateFunction()) + function_node.resolveAsAggregateFunction(std::move(aggregate_function), std::move(function_result_type)); + else if (function_node.isWindowFunction()) + function_node.resolveAsWindowFunction(std::move(aggregate_function), std::move(function_result_type)); } - static inline void resolveFunctionNodeAsFunction(FunctionNode & function_node, const String & function_name, const Data & data) + inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const { auto function_result_type = function_node.getResultType(); - auto function = FunctionFactory::instance().get(function_name, data.context); + auto function = FunctionFactory::instance().get(function_name, context); function_node.resolveAsFunction(function, std::move(function_result_type)); } + +private: + ContextPtr & context; }; } void CustomizeFunctionsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { - CustomizeFunctionsMatcher::Data data{context}; - CustomizeFunctionsMatcher::Visitor visitor(data); + CustomizeFunctionsVisitor visitor(context); visitor.visit(query_tree_node); } diff --git a/src/Analyzer/IfChainToMultiIfPass.cpp b/src/Analyzer/IfChainToMultiIfPass.cpp index 1b332e97433..4413c8f0cc8 100644 --- a/src/Analyzer/IfChainToMultiIfPass.cpp +++ b/src/Analyzer/IfChainToMultiIfPass.cpp @@ -12,17 +12,14 @@ namespace DB namespace { -class IfChainToMultiIfPassMatcher +class IfChainToMultiIfPassVisitor : public InDepthQueryTreeVisitor { public: - using Visitor = InDepthQueryTreeVisitor; + explicit IfChainToMultiIfPassVisitor(FunctionOverloadResolverPtr multi_if_function_ptr_) + : multi_if_function_ptr(std::move(multi_if_function_ptr_)) + {} - struct Data - { - FunctionOverloadResolverPtr multi_if_function_value; - }; - - static void visit(QueryTreeNodePtr & node, Data & data) + void visit(QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || function_node->getFunctionName() != "if" || function_node->getArguments().getNodes().size() != 3) @@ -58,23 +55,20 @@ public: return; auto multi_if_function = std::make_shared("multiIf"); - multi_if_function->resolveAsFunction(data.multi_if_function_value, std::make_shared()); + multi_if_function->resolveAsFunction(multi_if_function_ptr, std::make_shared()); multi_if_function->getArguments().getNodes() = std::move(multi_if_arguments); node = std::move(multi_if_function); } - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) - { - return true; - } +private: + FunctionOverloadResolverPtr multi_if_function_ptr; }; } void IfChainToMultiIfPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { - IfChainToMultiIfPassMatcher::Data data{FunctionFactory::instance().get("multiIf", context)}; - IfChainToMultiIfPassMatcher::Visitor visitor(data); + IfChainToMultiIfPassVisitor visitor(FunctionFactory::instance().get("multiIf", context)); visitor.visit(query_tree_node); } diff --git a/src/Analyzer/IfConstantConditionPass.cpp b/src/Analyzer/IfConstantConditionPass.cpp index 7aad78d78de..85c46b6b821 100644 --- a/src/Analyzer/IfConstantConditionPass.cpp +++ b/src/Analyzer/IfConstantConditionPass.cpp @@ -10,16 +10,10 @@ namespace DB namespace { -class IfConstantConditionMatcher +class IfConstantConditionVisitor : public InDepthQueryTreeVisitor { public: - using Visitor = InDepthQueryTreeVisitor; - - struct Data - { - }; - - static void visit(QueryTreeNodePtr & node, Data &) + void visitImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || (function_node->getFunctionName() != "if" && function_node->getFunctionName() != "multiIf")) @@ -49,19 +43,13 @@ public: else node = function_node->getArguments().getNodes()[2]; } - - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) - { - return true; - } }; } void IfConstantConditionPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) { - IfConstantConditionMatcher::Data data{}; - IfConstantConditionMatcher::Visitor visitor(data); + IfConstantConditionVisitor visitor; visitor.visit(query_tree_node); } diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index a1e71113263..fd0d8e9d10f 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -8,43 +8,63 @@ namespace DB { -/** Visit query tree in depth. - * Matcher need to define `visit`, `needChildVisit` methods and `Data` type. +/** Visitor that traverse query tree in depth. + * Derived class must implement `visitImpl` methods. + * Additionally subclass can control if child need to be visited using `needChildVisit` method, by + * default all node children are visited. + * By default visitor traverse tree from top to bottom, if bottom to top traverse is required subclass + * can override `shouldTraverseTopToBottom` method. + * + * Usage example: + * class FunctionsVisitor : public InDepthQueryTreeVisitor + * { + * void visitImpl(VisitQueryTreeNodeType & query_tree_node) + * { + * if (query_tree_node->getNodeType() == QueryTreeNodeType::FUNCTION) + * processFunctionNode(query_tree_node); + * } + * } */ -template +template class InDepthQueryTreeVisitor { public: - using Data = typename Matcher::Data; using VisitQueryTreeNodeType = std::conditional_t; - /// Initialize visitor with matchers data - explicit InDepthQueryTreeVisitor(Data & data_) - : data(data_) - {} + /// Return true if visitor should traverse tree top to bottom, false otherwise + bool shouldTraverseTopToBottom() const + { + return true; + } + + /// Return true if visitor should visit child, false otherwise + bool needChildVisit(VisitQueryTreeNodeType & parent [[maybe_unused]], VisitQueryTreeNodeType & child [[maybe_unused]]) + { + return true; + } - /// Visit query tree node void visit(VisitQueryTreeNodeType & query_tree_node) { - if constexpr (!top_to_bottom) + bool traverse_top_to_bottom = getDerived().shouldTraverseTopToBottom(); + if (!traverse_top_to_bottom) visitChildren(query_tree_node); - try - { - Matcher::visit(query_tree_node, data); - } - catch (Exception & e) - { - e.addMessage("While processing {}", query_tree_node->formatASTForErrorMessage()); - throw; - } + getDerived().visitImpl(query_tree_node); - if constexpr (top_to_bottom) + if (traverse_top_to_bottom) visitChildren(query_tree_node); } private: - Data & data; + Derived & getDerived() + { + return *static_cast(this); + } + + const Derived & getDerived() const + { + return *static_cast(this); + } void visitChildren(VisitQueryTreeNodeType & expression) { @@ -53,11 +73,7 @@ private: if (!child) continue; - bool need_visit_child = false; - if constexpr (need_child_accept_data) - need_visit_child = Matcher::needChildVisit(expression, child, data); - else - need_visit_child = Matcher::needChildVisit(expression, child); + bool need_visit_child = getDerived().needChildVisit(expression, child); if (need_visit_child) visit(child); @@ -65,7 +81,7 @@ private: } }; -template -using ConstInDepthQueryTreeVisitor = InDepthQueryTreeVisitor; +template +using ConstInDepthQueryTreeVisitor = InDepthQueryTreeVisitor; } diff --git a/src/Analyzer/MultiIfToIfPass.cpp b/src/Analyzer/MultiIfToIfPass.cpp index 5f6aa0143aa..0bd460ae5c7 100644 --- a/src/Analyzer/MultiIfToIfPass.cpp +++ b/src/Analyzer/MultiIfToIfPass.cpp @@ -10,17 +10,14 @@ namespace DB namespace { -class MultiIfToIfVisitorMatcher +class MultiIfToIfVisitor : public InDepthQueryTreeVisitor { public: - using Visitor = InDepthQueryTreeVisitor; + explicit MultiIfToIfVisitor(FunctionOverloadResolverPtr if_function_ptr_) + : if_function_ptr(if_function_ptr_) + {} - struct Data - { - FunctionOverloadResolverPtr if_function_overload_resolver; - }; - - static void visit(QueryTreeNodePtr & node, Data & data) + void visitImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || function_node->getFunctionName() != "multiIf") @@ -30,21 +27,18 @@ public: return; auto result_type = function_node->getResultType(); - function_node->resolveAsFunction(data.if_function_overload_resolver, result_type); + function_node->resolveAsFunction(if_function_ptr, result_type); } - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) - { - return true; - } +private: + FunctionOverloadResolverPtr if_function_ptr; }; } void MultiIfToIfPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { - MultiIfToIfVisitorMatcher::Data data{FunctionFactory::instance().get("if", context)}; - MultiIfToIfVisitorMatcher::Visitor visitor(data); + MultiIfToIfVisitor visitor(FunctionFactory::instance().get("if", context)); visitor.visit(query_tree_node); } diff --git a/src/Analyzer/NormalizeCountVariantsPass.cpp b/src/Analyzer/NormalizeCountVariantsPass.cpp index bdba4a73ae9..0c467592532 100644 --- a/src/Analyzer/NormalizeCountVariantsPass.cpp +++ b/src/Analyzer/NormalizeCountVariantsPass.cpp @@ -12,16 +12,10 @@ namespace DB namespace { -class NormalizeCountVariantsMatcher +class NormalizeCountVariantsVisitor : public InDepthQueryTreeVisitor { public: - using Visitor = InDepthQueryTreeVisitor; - - struct Data - { - }; - - static void visit(QueryTreeNodePtr & node, Data &) + void visitImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || !function_node->isAggregateFunction() || (function_node->getFunctionName() != "count" && function_node->getFunctionName() != "sum")) @@ -51,19 +45,13 @@ public: function_node->getArguments().getNodes().clear(); } } - - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) - { - return true; - } }; } void NormalizeCountVariantsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) { - NormalizeCountVariantsMatcher::Data data{}; - NormalizeCountVariantsMatcher::Visitor visitor(data); + NormalizeCountVariantsVisitor visitor; visitor.visit(query_tree_node); } diff --git a/src/Analyzer/OrderByTupleEliminationPass.cpp b/src/Analyzer/OrderByTupleEliminationPass.cpp index 2d5d9aa3bad..d25e7ee39e1 100644 --- a/src/Analyzer/OrderByTupleEliminationPass.cpp +++ b/src/Analyzer/OrderByTupleEliminationPass.cpp @@ -12,16 +12,10 @@ namespace DB namespace { -class OrderByTupleEliminationMatcher +class OrderByTupleEliminationVisitor : public InDepthQueryTreeVisitor { public: - using Visitor = InDepthQueryTreeVisitor; - - struct Data - { - }; - - static void visit(QueryTreeNodePtr & node, Data &) + void visitImpl(QueryTreeNodePtr & node) { auto * query_node = node->as(); if (!query_node || !query_node->hasOrderBy()) @@ -52,19 +46,13 @@ public: query_node->getOrderBy().getNodes() = std::move(result_nodes); } - - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) - { - return true; - } }; } void OrderByTupleEliminationPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) { - OrderByTupleEliminationMatcher::Data data{}; - OrderByTupleEliminationMatcher::Visitor visitor(data); + OrderByTupleEliminationVisitor visitor; visitor.visit(query_tree_node); } diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 9d0fd9bd6ac..04d636c5c58 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -804,6 +804,7 @@ struct IdentifierResolveScope * There are no easy solution here, without trying to make full featured expression resolution at this stage. * Example: * WITH (x -> x + 1) AS id, id AS id_1, id_1 AS id_2 SELECT id_2(1); + * Example: SELECT a, b AS a, b AS c, 1 AS c; * * It is client responsibility after resolving identifier node with alias, make following actions: * 1. If identifier node was resolved in function scope, remove alias from scope expression map. @@ -811,26 +812,23 @@ struct IdentifierResolveScope * * That way we separate alias map initialization and expressions resolution. */ -class QueryExpressionsAliasVisitorMatcher +class QueryExpressionsAliasVisitor : public InDepthQueryTreeVisitor { public: - using Visitor = InDepthQueryTreeVisitor; + explicit QueryExpressionsAliasVisitor(IdentifierResolveScope & scope_) + : scope(scope_) + {} - struct Data + void visitImpl(QueryTreeNodePtr & node) { - IdentifierResolveScope & scope; - }; - - static void visit(QueryTreeNodePtr & node, Data & data) - { - updateAliasesIfNeeded(data, node, false); + updateAliasesIfNeeded(node, false /*is_lambda_node*/); } - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child, Data & data) + bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child) { if (auto * lambda_node = child->as()) { - updateAliasesIfNeeded(data, child, true); + updateAliasesIfNeeded(child, true /*is_lambda_node*/); return false; } else if (auto * query_tree_node = child->as()) @@ -838,7 +836,7 @@ public: if (query_tree_node->isCTE()) return false; - updateAliasesIfNeeded(data, child, false); + updateAliasesIfNeeded(child, false /*is_lambda_node*/); return false; } else if (auto * union_node = child->as()) @@ -846,60 +844,57 @@ public: if (union_node->isCTE()) return false; - updateAliasesIfNeeded(data, child, false); + updateAliasesIfNeeded(child, false /*is_lambda_node*/); return false; } return true; } private: - static void updateAliasesIfNeeded(Data & data, const QueryTreeNodePtr & node, bool function_node) + void updateAliasesIfNeeded(const QueryTreeNodePtr & node, bool is_lambda_node) { if (!node->hasAlias()) return; const auto & alias = node->getAlias(); - if (function_node) + if (is_lambda_node) { - if (data.scope.alias_name_to_expression_node.contains(alias)) - data.scope.nodes_with_duplicated_aliases.insert(node); + if (scope.alias_name_to_expression_node.contains(alias)) + scope.nodes_with_duplicated_aliases.insert(node); - auto [_, inserted] = data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); + auto [_, inserted] = scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); if (!inserted) - data.scope.nodes_with_duplicated_aliases.insert(node); + scope.nodes_with_duplicated_aliases.insert(node); return; } - if (data.scope.alias_name_to_lambda_node.contains(alias)) - data.scope.nodes_with_duplicated_aliases.insert(node); + if (scope.alias_name_to_lambda_node.contains(alias)) + scope.nodes_with_duplicated_aliases.insert(node); - auto [_, inserted] = data.scope.alias_name_to_expression_node.insert(std::make_pair(alias, node)); + auto [_, inserted] = scope.alias_name_to_expression_node.insert(std::make_pair(alias, node)); if (!inserted) - data.scope.nodes_with_duplicated_aliases.insert(node); + scope.nodes_with_duplicated_aliases.insert(node); /// If node is identifier put it also in scope alias name to lambda node map if (node->getNodeType() == QueryTreeNodeType::IDENTIFIER) - data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); + scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); } + + IdentifierResolveScope & scope; }; -using QueryExpressionsAliasVisitor = QueryExpressionsAliasVisitorMatcher::Visitor; - -class TableExpressionsAliasVisitorMatcher +class TableExpressionsAliasVisitor : public InDepthQueryTreeVisitor { public: - using Visitor = InDepthQueryTreeVisitor; + explicit TableExpressionsAliasVisitor(IdentifierResolveScope & scope_) + : scope(scope_) + {} - struct Data + void visitImpl(QueryTreeNodePtr & node) { - IdentifierResolveScope & scope; - }; - - static void visit(QueryTreeNodePtr & node, Data & data) - { - updateAliasesIfNeeded(data, node); + updateAliasesIfNeeded(node); } static bool needChildVisit(const QueryTreeNodePtr & node, const QueryTreeNodePtr & child) @@ -928,22 +923,22 @@ public: } private: - static void updateAliasesIfNeeded(Data & data, const QueryTreeNodePtr & node) + void updateAliasesIfNeeded(const QueryTreeNodePtr & node) { if (!node->hasAlias()) return; const auto & node_alias = node->getAlias(); - auto [_, inserted] = data.scope.alias_name_to_table_expression_node.emplace(node_alias, node); + auto [_, inserted] = scope.alias_name_to_table_expression_node.emplace(node_alias, node); if (!inserted) throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, "Multiple table expressions with same alias {}. In scope {}", node_alias, - data.scope.scope_node->formatASTForErrorMessage()); + scope.scope_node->formatASTForErrorMessage()); } -}; -using TableExpressionsAliasVisitor = TableExpressionsAliasVisitorMatcher::Visitor; + IdentifierResolveScope & scope; +}; class QueryAnalyzer { @@ -3367,8 +3362,7 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod scope.scope_node->formatASTForErrorMessage()); /// Initialize aliases in lambda scope - QueryExpressionsAliasVisitor::Data data{scope}; - QueryExpressionsAliasVisitor visitor(data); + QueryExpressionsAliasVisitor visitor(scope); visitor.visit(lambda.getExpression()); /** Replace lambda arguments with new arguments. @@ -3384,8 +3378,8 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod auto & lambda_argument_node_typed = lambda_argument_node->as(); const auto & lambda_argument_name = lambda_argument_node_typed.getIdentifier().getFullName(); - bool has_expression_node = data.scope.alias_name_to_expression_node.contains(lambda_argument_name); - bool has_alias_node = data.scope.alias_name_to_lambda_node.contains(lambda_argument_name); + bool has_expression_node = scope.alias_name_to_expression_node.contains(lambda_argument_name); + bool has_alias_node = scope.alias_name_to_lambda_node.contains(lambda_argument_name); if (has_expression_node || has_alias_node) { @@ -4783,8 +4777,7 @@ void QueryAnalyzer::initializeTableExpressionColumns(const QueryTreeNodePtr & ta alias_column_resolve_scope.column_name_to_column_node = std::move(column_name_to_column_node); /// Initialize aliases in alias column scope - QueryExpressionsAliasVisitor::Data data{alias_column_resolve_scope}; - QueryExpressionsAliasVisitor visitor(data); + QueryExpressionsAliasVisitor visitor(alias_column_resolve_scope); visitor.visit(alias_column_to_resolve->getExpression()); @@ -5074,18 +5067,15 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, scope.table_expressions_in_resolve_process.erase(join_tree_node.get()); } -class ValidateGroupByColumnsMatcher +class ValidateGroupByColumnsVisitor : public ConstInDepthQueryTreeVisitor { public: - using Visitor = ConstInDepthQueryTreeVisitor; + ValidateGroupByColumnsVisitor(const QueryTreeNodes & group_by_keys_nodes_, const IdentifierResolveScope & scope_) + : group_by_keys_nodes(group_by_keys_nodes_) + , scope(scope_) + {} - struct Data - { - const QueryTreeNodes & group_by_keys_nodes; - const IdentifierResolveScope & scope; - }; - - static void visit(const QueryTreeNodePtr & node, Data & data) + void visitImpl(const QueryTreeNodePtr & node) { auto query_tree_node_type = node->getNodeType(); if (query_tree_node_type == QueryTreeNodeType::CONSTANT || @@ -5101,7 +5091,7 @@ public: { bool found_argument_in_group_by_keys = false; - for (const auto & group_by_key_node : data.group_by_keys_nodes) + for (const auto & group_by_key_node : group_by_keys_nodes) { if (grouping_function_arguments_node->isEqual(*group_by_key_node)) { @@ -5114,7 +5104,7 @@ public: throw Exception(ErrorCodes::NOT_AN_AGGREGATE, "GROUPING function argument {} is not in GROUP BY. In scope {}", grouping_function_arguments_node->formatASTForErrorMessage(), - data.scope.scope_node->formatASTForErrorMessage()); + scope.scope_node->formatASTForErrorMessage()); } return; @@ -5128,7 +5118,7 @@ public: if (column_node_source->getNodeType() == QueryTreeNodeType::LAMBDA) return; - for (const auto & group_by_key_node : data.group_by_keys_nodes) + for (const auto & group_by_key_node : group_by_keys_nodes) { if (node->isEqual(*group_by_key_node)) return; @@ -5146,10 +5136,10 @@ public: throw Exception(ErrorCodes::NOT_AN_AGGREGATE, "Column {} is not under aggregate function and not in GROUP BY. In scope {}", column_name, - data.scope.scope_node->formatASTForErrorMessage()); + scope.scope_node->formatASTForErrorMessage()); } - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node, Data & data) + bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { auto * child_function_node = child_node->as(); if (child_function_node) @@ -5157,7 +5147,7 @@ public: if (child_function_node->isAggregateFunction()) return false; - for (const auto & group_by_key_node : data.group_by_keys_nodes) + for (const auto & group_by_key_node : group_by_keys_nodes) { if (child_node->isEqual(*group_by_key_node)) return false; @@ -5166,44 +5156,41 @@ public: return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); } + +private: + const QueryTreeNodes & group_by_keys_nodes; + const IdentifierResolveScope & scope; }; -using ValidateGroupByColumnsVisitor = ValidateGroupByColumnsMatcher::Visitor; - -class ValidateGroupingFunctionNodesMatcher +class ValidateGroupingFunctionNodesVisitor : public ConstInDepthQueryTreeVisitor { public: - using Visitor = ConstInDepthQueryTreeVisitor; + explicit ValidateGroupingFunctionNodesVisitor(String assert_no_grouping_function_place_message_) + : assert_no_grouping_function_place_message(std::move(assert_no_grouping_function_place_message_)) + {} - struct Data - { - String assert_no_grouping_function_place_message; - }; - - static void visit(const QueryTreeNodePtr & node, Data & data) + void visitImpl(const QueryTreeNodePtr & node) { auto * function_node = node->as(); if (function_node && function_node->getFunctionName() == "grouping") throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, "GROUPING function {} is found {} in query", function_node->formatASTForErrorMessage(), - data.assert_no_grouping_function_place_message); + assert_no_grouping_function_place_message); } static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; } -}; -using ValidateGroupingFunctionNodesVisitor = ValidateGroupingFunctionNodesMatcher::Visitor; +private: + String assert_no_grouping_function_place_message; +}; void assertNoGroupingFunction(const QueryTreeNodePtr & node, const String & assert_no_grouping_function_place_message) { - ValidateGroupingFunctionNodesVisitor::Data data; - data.assert_no_grouping_function_place_message = assert_no_grouping_function_place_message; - - ValidateGroupingFunctionNodesVisitor visitor(data); + ValidateGroupingFunctionNodesVisitor visitor(assert_no_grouping_function_place_message); visitor.visit(node); } @@ -5237,9 +5224,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier auto & query_node_typed = query_node->as(); /// Initialize aliases in query node scope - - QueryExpressionsAliasVisitor::Data data{scope}; - QueryExpressionsAliasVisitor visitor(data); + QueryExpressionsAliasVisitor visitor(scope); if (query_node_typed.hasWith()) visitor.visit(query_node_typed.getWithNode()); @@ -5294,12 +5279,12 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier continue; const auto & cte_name = subquery_node->getCTEName(); - auto [_, inserted] = data.scope.cte_name_to_query_node.emplace(cte_name, node); + auto [_, inserted] = scope.cte_name_to_query_node.emplace(cte_name, node); if (!inserted) throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, "CTE with name {} already exists. In scope {}", cte_name, - data.scope.scope_node->formatASTForErrorMessage()); + scope.scope_node->formatASTForErrorMessage()); } std::erase_if(with_nodes, [](const QueryTreeNodePtr & node) @@ -5340,9 +5325,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.getJoinTree()) { - TableExpressionsAliasVisitor::Data table_expressions_visitor_data{scope}; - TableExpressionsAliasVisitor table_expressions_visitor(table_expressions_visitor_data); - + TableExpressionsAliasVisitor table_expressions_visitor(scope); table_expressions_visitor.visit(query_node_typed.getJoinTree()); initializeQueryJoinTreeNode(query_node_typed.getJoinTree(), scope); @@ -5463,8 +5446,8 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (!it->second->isEqual(*node)) throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, "Multiple expressions {} and {} for alias {}. In scope {}", - node->dumpTree(), - it->second->dumpTree(), + node->formatASTForErrorMessage(), + it->second->formatASTForErrorMessage(), node_alias, scope.scope_node->formatASTForErrorMessage()); } @@ -5588,16 +5571,15 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (has_aggregation) { - ValidateGroupByColumnsVisitor::Data validate_group_by_visitor_data {group_by_keys_nodes, scope}; - ValidateGroupByColumnsVisitor validate_group_by_visitor(validate_group_by_visitor_data); + ValidateGroupByColumnsVisitor validate_group_by_columns_visitor(group_by_keys_nodes, scope); if (query_node_typed.hasHaving()) - validate_group_by_visitor.visit(query_node_typed.getHaving()); + validate_group_by_columns_visitor.visit(query_node_typed.getHaving()); if (query_node_typed.hasOrderBy()) - validate_group_by_visitor.visit(query_node_typed.getOrderByNode()); + validate_group_by_columns_visitor.visit(query_node_typed.getOrderByNode()); - validate_group_by_visitor.visit(query_node_typed.getProjectionNode()); + validate_group_by_columns_visitor.visit(query_node_typed.getProjectionNode()); } if (context->getSettingsRef().group_by_use_nulls) diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index f99e87f08d9..534999419ae 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -42,7 +42,6 @@ #include #include #include -#include #include diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index bafc2cc2644..bd6acb90da2 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -5,57 +5,15 @@ #include #include -#include -#include -#include -#include -#include - -#include - -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include -#include -#include -#include -#include -#include - -#include -#include -#include +#include #include -#include -#include #include -#include -#include -#include -#include -#include -#include #include #include -#include -#include -#include -#include namespace DB { diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 92cd63999b5..65348c38d4a 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -22,17 +22,14 @@ namespace ErrorCodes namespace { -class CollectSetsMatcher +class CollectSetsVisitor : public ConstInDepthQueryTreeVisitor { public: - using Visitor = ConstInDepthQueryTreeVisitor; + explicit CollectSetsVisitor(const PlannerContext & planner_context_) + : planner_context(planner_context_) + {} - struct Data - { - const PlannerContext & planner_context; - }; - - static void visit(const QueryTreeNodePtr & node, Data & data) + void visitImpl(const QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || !isNameOfInFunction(function_node->getFunctionName())) @@ -42,7 +39,6 @@ public: auto in_second_argument = function_node->getArguments().getNodes().at(1); auto in_second_argument_node_type = in_second_argument->getNodeType(); - const auto & planner_context = data.planner_context; const auto & global_planner_context = planner_context.getGlobalPlannerContext(); const auto & settings = planner_context.getQueryContext()->getSettingsRef(); @@ -93,16 +89,16 @@ public: { return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); } -}; -using CollectSetsVisitor = CollectSetsMatcher::Visitor; +private: + const PlannerContext & planner_context; +}; } void collectSets(const QueryTreeNodePtr & node, const PlannerContext & planner_context) { - CollectSetsVisitor::Data data {planner_context}; - CollectSetsVisitor visitor(data); + CollectSetsVisitor visitor(planner_context); visitor.visit(node); } diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index cf7ead918d4..30ccc541507 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -23,17 +23,14 @@ namespace ErrorCodes namespace { -class CollectSourceColumnsMatcher +class CollectSourceColumnsVisitor : public InDepthQueryTreeVisitor { public: - using Visitor = InDepthQueryTreeVisitor; + explicit CollectSourceColumnsVisitor(PlannerContext & planner_context_) + : planner_context(planner_context_) + {} - struct Data - { - PlannerContext & planner_context; - }; - - static void visit(QueryTreeNodePtr & node, Data & data) + void visitImpl(QueryTreeNodePtr & node) { auto * column_node = node->as(); if (!column_node) @@ -50,14 +47,14 @@ public: if (column_node->hasExpression() && column_source_node->getNodeType() == QueryTreeNodeType::JOIN) return; - auto & table_expression_data = data.planner_context.getOrCreateTableExpressionData(column_source_node); + auto & table_expression_data = planner_context.getOrCreateTableExpressionData(column_source_node); if (column_node->hasExpression()) { /// Replace ALIAS column with expression table_expression_data.addAliasColumnName(column_node->getColumnName()); node = column_node->getExpression(); - visit(node, data); + visitImpl(node); return; } @@ -73,7 +70,7 @@ public: if (column_already_exists) return; - auto column_identifier = data.planner_context.getGlobalPlannerContext()->createColumnIdentifier(node); + auto column_identifier = planner_context.getGlobalPlannerContext()->createColumnIdentifier(node); table_expression_data.addColumn(column_node->getColumn(), column_identifier); } @@ -81,9 +78,10 @@ public: { return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); } -}; -using CollectSourceColumnsVisitor = CollectSourceColumnsMatcher::Visitor; +private: + PlannerContext & planner_context; +}; } @@ -111,8 +109,7 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContext & throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Remote storages are not supported"); } - CollectSourceColumnsVisitor::Data data {planner_context}; - CollectSourceColumnsVisitor collect_source_columns_visitor(data); + CollectSourceColumnsVisitor collect_source_columns_visitor(planner_context); collect_source_columns_visitor.visit(query_node); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index a3fb81d707a..88e80df5c6e 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -45,7 +45,6 @@ #include #include #include -#include #include #include diff --git a/src/Planner/PlannerAggregation.cpp b/src/Planner/PlannerAggregation.cpp index 2480e47bb82..569b8525b3b 100644 --- a/src/Planner/PlannerAggregation.cpp +++ b/src/Planner/PlannerAggregation.cpp @@ -32,64 +32,53 @@ enum class GroupByKind GROUPING_SETS }; -class GroupingFunctionResolveMatcher +class GroupingFunctionResolveVisitor : public InDepthQueryTreeVisitor { public: - using Visitor = InDepthQueryTreeVisitor; - - struct Data + GroupingFunctionResolveVisitor(GroupByKind group_by_kind_, + const Names & aggregation_keys_, + const GroupingSetsParamsList & grouping_sets_parameters_list_, + const PlannerContext & planner_context_) + : group_by_kind(group_by_kind_) + , planner_context(planner_context_) { - Data(GroupByKind group_by_kind_, - const Names & aggregation_keys_, - const GroupingSetsParamsList & grouping_sets_parameters_list_, - const PlannerContext & planner_context_) - : group_by_kind(group_by_kind_) - , planner_context(planner_context_) + size_t aggregation_keys_size = aggregation_keys_.size(); + for (size_t i = 0; i < aggregation_keys_size; ++i) + aggegation_key_to_index.emplace(aggregation_keys_[i], i); + + for (const auto & grouping_sets_parameter : grouping_sets_parameters_list_) { - size_t aggregation_keys_size = aggregation_keys_.size(); - for (size_t i = 0; i < aggregation_keys_size; ++i) - aggegation_key_to_index.emplace(aggregation_keys_[i], i); + grouping_sets_keys_indices.emplace_back(); + auto & grouping_set_keys_indices = grouping_sets_keys_indices.back(); - for (const auto & grouping_sets_parameter : grouping_sets_parameters_list_) + for (const auto & used_key : grouping_sets_parameter.used_keys) { - grouping_sets_keys_indices.emplace_back(); - auto & grouping_set_keys_indices = grouping_sets_keys_indices.back(); + auto aggregation_key_index_it = aggegation_key_to_index.find(used_key); + if (aggregation_key_index_it == aggegation_key_to_index.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Aggregation key {} in GROUPING SETS is not found in GROUP BY keys"); - for (const auto & used_key : grouping_sets_parameter.used_keys) - { - auto aggregation_key_index_it = aggegation_key_to_index.find(used_key); - if (aggregation_key_index_it == aggegation_key_to_index.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Aggregation key {} in GROUPING SETS is not found in GROUP BY keys"); - - grouping_set_keys_indices.push_back(aggregation_key_index_it->second); - } + grouping_set_keys_indices.push_back(aggregation_key_index_it->second); } } + } - GroupByKind group_by_kind; - std::unordered_map aggegation_key_to_index; - // Indexes of aggregation keys used in each grouping set (only for GROUP BY GROUPING SETS) - ColumnNumbersList grouping_sets_keys_indices; - const PlannerContext & planner_context; - }; - - static void visit(const QueryTreeNodePtr & node, Data & data) + void visitImpl(const QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || function_node->getFunctionName() != "grouping") return; - size_t aggregation_keys_size = data.aggegation_key_to_index.size(); + size_t aggregation_keys_size = aggegation_key_to_index.size(); ColumnNumbers arguments_indexes; for (const auto & argument : function_node->getArguments().getNodes()) { - String action_node_name = calculateActionNodeName(argument, data.planner_context); + String action_node_name = calculateActionNodeName(argument, planner_context); - auto it = data.aggegation_key_to_index.find(action_node_name); - if (it == data.aggegation_key_to_index.end()) + auto it = aggegation_key_to_index.find(action_node_name); + if (it == aggegation_key_to_index.end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of GROUPING function {} is not a part of GROUP BY clause", argument->formatASTForErrorMessage()); @@ -101,9 +90,9 @@ public: auto grouping_set_argument_column = std::make_shared(NameAndTypePair{"__grouping_set", std::make_shared()}, column_source); function_node->getArguments().getNodes().clear(); - bool force_grouping_standard_compatibility = data.planner_context.getQueryContext()->getSettingsRef().force_grouping_standard_compatibility; + bool force_grouping_standard_compatibility = planner_context.getQueryContext()->getSettingsRef().force_grouping_standard_compatibility; - switch (data.group_by_kind) + switch (group_by_kind) { case GroupByKind::ORDINARY: { @@ -130,7 +119,7 @@ public: } case GroupByKind::GROUPING_SETS: { - auto grouping_grouping_sets_function = std::make_shared(arguments_indexes, data.grouping_sets_keys_indices, force_grouping_standard_compatibility); + auto grouping_grouping_sets_function = std::make_shared(arguments_indexes, grouping_sets_keys_indices, force_grouping_standard_compatibility); auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_grouping_sets_function)); function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); function_node->getArguments().getNodes().push_back(std::move(grouping_set_argument_column)); @@ -143,9 +132,14 @@ public: { return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); } -}; -using GroupingFunctionResolveVisitor = GroupingFunctionResolveMatcher::Visitor; +private: + GroupByKind group_by_kind; + std::unordered_map aggegation_key_to_index; + // Indexes of aggregation keys used in each grouping set (only for GROUP BY GROUPING SETS) + ColumnNumbersList grouping_sets_keys_indices; + const PlannerContext & planner_context; +}; void resolveGroupingFunctions(QueryTreeNodePtr & node, GroupByKind group_by_kind, @@ -153,8 +147,7 @@ void resolveGroupingFunctions(QueryTreeNodePtr & node, const GroupingSetsParamsList & grouping_sets_parameters_list, const PlannerContext & planner_context) { - GroupingFunctionResolveVisitor::Data data {group_by_kind, aggregation_keys, grouping_sets_parameters_list, planner_context}; - GroupingFunctionResolveVisitor visitor(data); + GroupingFunctionResolveVisitor visitor(group_by_kind, aggregation_keys, grouping_sets_parameters_list, planner_context); visitor.visit(node); } diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index a1ae572727f..c977c9a2090 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -1,6 +1,5 @@ #include -#include #include #include From 6ec54744665a9f0410772baf2de9fc3bafedc528 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 10 Oct 2022 12:25:58 +0200 Subject: [PATCH 128/188] Updated IQueryTreeNode interface with weak pointers array --- src/Analyzer/ArrayJoinNode.cpp | 10 +- src/Analyzer/ArrayJoinNode.h | 6 +- src/Analyzer/ColumnNode.cpp | 53 +--- src/Analyzer/ColumnNode.h | 27 +- src/Analyzer/ColumnTransformers.cpp | 47 ++-- src/Analyzer/ColumnTransformers.h | 18 +- src/Analyzer/ConstantNode.cpp | 10 +- src/Analyzer/ConstantNode.h | 6 +- src/Analyzer/FunctionNode.cpp | 30 +-- src/Analyzer/FunctionNode.h | 6 +- src/Analyzer/IQueryTreeNode.cpp | 246 ++++++++++++------- src/Analyzer/IQueryTreeNode.h | 109 ++++---- src/Analyzer/IdentifierNode.cpp | 10 +- src/Analyzer/IdentifierNode.h | 6 +- src/Analyzer/InterpolateNode.cpp | 10 +- src/Analyzer/InterpolateNode.h | 6 +- src/Analyzer/JoinNode.cpp | 10 +- src/Analyzer/JoinNode.h | 6 +- src/Analyzer/LambdaNode.cpp | 10 +- src/Analyzer/LambdaNode.h | 6 +- src/Analyzer/ListNode.cpp | 10 +- src/Analyzer/ListNode.h | 6 +- src/Analyzer/MatcherNode.cpp | 26 +- src/Analyzer/MatcherNode.h | 6 +- src/Analyzer/QueryNode.cpp | 40 +-- src/Analyzer/QueryNode.h | 6 +- src/Analyzer/SortNode.cpp | 10 +- src/Analyzer/SortNode.h | 7 +- src/Analyzer/TableFunctionNode.cpp | 24 +- src/Analyzer/TableFunctionNode.h | 6 +- src/Analyzer/TableNode.cpp | 10 +- src/Analyzer/TableNode.h | 6 +- src/Analyzer/UnionNode.cpp | 32 +-- src/Analyzer/UnionNode.h | 6 +- src/Analyzer/WindowNode.cpp | 16 +- src/Analyzer/WindowNode.h | 6 +- src/Analyzer/tests/gtest_query_tree_node.cpp | 10 +- 37 files changed, 451 insertions(+), 408 deletions(-) diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp index 1504a7ab7a2..2157b5edf6f 100644 --- a/src/Analyzer/ArrayJoinNode.cpp +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -42,6 +42,11 @@ void ArrayJoinNode::updateTreeHashImpl(HashState & state) const state.update(is_left); } +QueryTreeNodePtr ArrayJoinNode::cloneImpl() const +{ + return std::make_shared(getTableExpression(), getJoinExpressionsNode(), is_left); +} + ASTPtr ArrayJoinNode::toASTImpl() const { auto array_join_ast = std::make_shared(); @@ -63,9 +68,4 @@ ASTPtr ArrayJoinNode::toASTImpl() const return tables_in_select_query_ast; } -QueryTreeNodePtr ArrayJoinNode::cloneImpl() const -{ - return std::make_shared(getTableExpression(), getJoinExpressionsNode(), is_left); -} - } diff --git a/src/Analyzer/ArrayJoinNode.h b/src/Analyzer/ArrayJoinNode.h index d2e0cd6e6eb..7ca61b444e2 100644 --- a/src/Analyzer/ArrayJoinNode.h +++ b/src/Analyzer/ArrayJoinNode.h @@ -90,15 +90,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: bool is_left = false; diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index 231acb695cd..4d0d349dabb 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -16,24 +16,22 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_) - : IQueryTreeNode(children_size) - , column(std::move(column_)) - , column_source(std::move(column_source_)) -{ -} - ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_) - : IQueryTreeNode(children_size) + : IQueryTreeNode(children_size, weak_pointers_size) , column(std::move(column_)) - , column_source(std::move(column_source_)) { children[expression_child_index] = std::move(expression_node_); + getSourceWeakPointer() = std::move(column_source_); +} + +ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_) + : ColumnNode(std::move(column_), nullptr /*expression_node*/, std::move(column_source_)) +{ } QueryTreeNodePtr ColumnNode::getColumnSource() const { - auto lock = column_source.lock(); + auto lock = getSourceWeakPointer().lock(); if (!lock) throw Exception(ErrorCodes::LOGICAL_ERROR, "Column {} {} query tree node does not have valid source node", @@ -45,7 +43,7 @@ QueryTreeNodePtr ColumnNode::getColumnSource() const QueryTreeNodePtr ColumnNode::getColumnSourceOrNull() const { - return column_source.lock(); + return getSourceWeakPointer().lock(); } void ColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t indent) const @@ -57,7 +55,7 @@ void ColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t buffer << ", column_name: " << column.name << ", result_type: " << column.type->getName(); - auto column_source_ptr = column_source.lock(); + auto column_source_ptr = getSourceWeakPointer().lock(); if (column_source_ptr) buffer << ", source_id: " << state.getNodeId(column_source_ptr.get()); @@ -73,20 +71,7 @@ void ColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t bool ColumnNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); - if (column != rhs_typed.column) - return false; - - auto source_ptr = column_source.lock(); - auto rhs_source_ptr = rhs_typed.column_source.lock(); - - if (!source_ptr && !rhs_source_ptr) - return true; - else if (source_ptr && !rhs_source_ptr) - return false; - else if (!source_ptr && rhs_source_ptr) - return false; - - return source_ptr->isEqualImpl(*rhs_source_ptr); + return column == rhs_typed.column; } void ColumnNode::updateTreeHashImpl(HashState & hash_state) const @@ -97,25 +82,11 @@ void ColumnNode::updateTreeHashImpl(HashState & hash_state) const const auto & column_type_name = column.type->getName(); hash_state.update(column_type_name.size()); hash_state.update(column_type_name); - - auto column_source_ptr = column_source.lock(); - if (column_source_ptr) - column_source_ptr->updateTreeHashImpl(hash_state); } QueryTreeNodePtr ColumnNode::cloneImpl() const { - return std::make_shared(column, column_source); -} - -void ColumnNode::getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update) -{ - /** This method is called on node returned from `cloneImpl`. Check IQueryTreeNode.h interface. - * old pointer is current column source pointer. - * update place is address of column source. - */ - const auto * old_pointer = getColumnSource().get(); - pointers_to_update.emplace_back(old_pointer, &column_source); + return std::make_shared(column, getColumnSource()); } ASTPtr ColumnNode::toASTImpl() const diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index f22feb30d6e..212285589d0 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -37,12 +37,12 @@ using ColumnNodePtr = std::shared_ptr; class ColumnNode final : public IQueryTreeNode { public: - /// Construct column node with column name, type and column source weak pointer. - ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_); - /// Construct expression column node with column name, type, column expression and column source weak pointer. ColumnNode(NameAndTypePair column_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_); + /// Construct column node with column name, type and column source weak pointer. + ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_); + /// Get column const NameAndTypePair & getColumn() const { @@ -122,23 +122,34 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t indent) const override; +protected: + bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & hash_state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; - void getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update) override; + ASTPtr toASTImpl() const override; private: + const QueryTreeNodeWeakPtr & getSourceWeakPointer() const + { + return weak_pointers[source_weak_pointer_index]; + } + + QueryTreeNodeWeakPtr & getSourceWeakPointer() + { + return weak_pointers[source_weak_pointer_index]; + } + NameAndTypePair column; - QueryTreeNodeWeakPtr column_source; static constexpr size_t expression_child_index = 0; static constexpr size_t children_size = expression_child_index + 1; + + static constexpr size_t source_weak_pointer_index = 0; + static constexpr size_t weak_pointers_size = source_weak_pointer_index + 1; }; } diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp index 541d116bc59..ac65094ea05 100644 --- a/src/Analyzer/ColumnTransformers.cpp +++ b/src/Analyzer/ColumnTransformers.cpp @@ -84,8 +84,11 @@ void ApplyColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & { hash_state.update(static_cast(getTransformerType())); hash_state.update(static_cast(getApplyTransformerType())); +} - getExpressionNode()->updateTreeHash(hash_state); +QueryTreeNodePtr ApplyColumnTransformerNode::cloneImpl() const +{ + return std::make_shared(getExpressionNode()); } ASTPtr ApplyColumnTransformerNode::toASTImpl() const @@ -110,11 +113,6 @@ ASTPtr ApplyColumnTransformerNode::toASTImpl() const return ast_apply_transformer; } -QueryTreeNodePtr ApplyColumnTransformerNode::cloneImpl() const -{ - return std::make_shared(getExpressionNode()); -} - /// ExceptColumnTransformerNode implementation ExceptColumnTransformerNode::ExceptColumnTransformerNode(Names except_column_names_, bool is_strict_) @@ -219,6 +217,14 @@ void ExceptColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & } } +QueryTreeNodePtr ExceptColumnTransformerNode::cloneImpl() const +{ + if (except_transformer_type == ExceptColumnTransformerType::REGEXP) + return std::make_shared(column_matcher); + + return std::make_shared(except_column_names, is_strict); +} + ASTPtr ExceptColumnTransformerNode::toASTImpl() const { auto ast_except_transformer = std::make_shared(); @@ -236,14 +242,6 @@ ASTPtr ExceptColumnTransformerNode::toASTImpl() const return ast_except_transformer; } -QueryTreeNodePtr ExceptColumnTransformerNode::cloneImpl() const -{ - if (except_transformer_type == ExceptColumnTransformerType::REGEXP) - return std::make_shared(column_matcher); - - return std::make_shared(except_column_names, is_strict); -} - /// ReplaceColumnTransformerNode implementation ReplaceColumnTransformerNode::ReplaceColumnTransformerNode(const std::vector & replacements_, bool is_strict_) @@ -321,10 +319,19 @@ void ReplaceColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState const auto & replacement_name = replacements_names[i]; hash_state.update(replacement_name.size()); hash_state.update(replacement_name); - replacement_expressions_nodes[i]->updateTreeHash(hash_state); } } +QueryTreeNodePtr ReplaceColumnTransformerNode::cloneImpl() const +{ + auto result_replace_transformer = std::make_shared(std::vector{}, false); + + result_replace_transformer->is_strict = is_strict; + result_replace_transformer->replacements_names = replacements_names; + + return result_replace_transformer; +} + ASTPtr ReplaceColumnTransformerNode::toASTImpl() const { auto ast_replace_transformer = std::make_shared(); @@ -345,14 +352,4 @@ ASTPtr ReplaceColumnTransformerNode::toASTImpl() const return ast_replace_transformer; } -QueryTreeNodePtr ReplaceColumnTransformerNode::cloneImpl() const -{ - auto result_replace_transformer = std::make_shared(std::vector{}, false); - - result_replace_transformer->is_strict = is_strict; - result_replace_transformer->replacements_names = replacements_names; - - return result_replace_transformer; -} - } diff --git a/src/Analyzer/ColumnTransformers.h b/src/Analyzer/ColumnTransformers.h index 221888160a5..fa45f2deee9 100644 --- a/src/Analyzer/ColumnTransformers.h +++ b/src/Analyzer/ColumnTransformers.h @@ -134,15 +134,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: ApplyColumnTransformerType apply_transformer_type = ApplyColumnTransformerType::LAMBDA; @@ -213,15 +213,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: ExceptColumnTransformerType except_transformer_type; Names except_column_names; @@ -291,15 +291,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: Names replacements_names; bool is_strict = false; diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index 4a292d5501b..b7de2acb5a4 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -58,14 +58,14 @@ void ConstantNode::updateTreeHashImpl(HashState & hash_state) const hash_state.update(value_string); } -ASTPtr ConstantNode::toASTImpl() const -{ - return std::make_shared(constant_value->getValue()); -} - QueryTreeNodePtr ConstantNode::cloneImpl() const { return std::make_shared(constant_value); } +ASTPtr ConstantNode::toASTImpl() const +{ + return std::make_shared(constant_value->getValue()); +} + } diff --git a/src/Analyzer/ConstantNode.h b/src/Analyzer/ConstantNode.h index 25931e848d5..29e8cd25532 100644 --- a/src/Analyzer/ConstantNode.h +++ b/src/Analyzer/ConstantNode.h @@ -61,17 +61,17 @@ public: return constant_value->getType(); } -protected: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & hash_state) const override; - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: ConstantValuePtr constant_value; String value_string; diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index feea942d5a2..c5bb3d5c56d 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -167,6 +167,21 @@ void FunctionNode::updateTreeHashImpl(HashState & hash_state) const } } +QueryTreeNodePtr FunctionNode::cloneImpl() const +{ + auto result_function = std::make_shared(function_name); + + /** This is valid for clone method to reuse same function pointers + * because ordinary functions or aggregate functions must be stateless. + */ + result_function->function = function; + result_function->aggregate_function = aggregate_function; + result_function->result_type = result_type; + result_function->constant_value = constant_value; + + return result_function; +} + ASTPtr FunctionNode::toASTImpl() const { auto function_ast = std::make_shared(); @@ -197,19 +212,4 @@ ASTPtr FunctionNode::toASTImpl() const return function_ast; } -QueryTreeNodePtr FunctionNode::cloneImpl() const -{ - auto result_function = std::make_shared(function_name); - - /** This is valid for clone method to reuse same function pointers - * because ordinary functions or aggregate functions must be stateless. - */ - result_function->function = function; - result_function->aggregate_function = aggregate_function; - result_function->result_type = result_type; - result_function->constant_value = constant_value; - - return result_function; -} - } diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index d8dd869c25e..40ab9a77d4d 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -206,15 +206,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & hash_state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: String function_name; FunctionOverloadResolverPtr function; diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index f6354c7f9a6..178abd4550b 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -42,59 +42,85 @@ const char * toString(QueryTreeNodeType type) } } +IQueryTreeNode::IQueryTreeNode(size_t children_size, size_t weak_pointers_size) +{ + children.resize(children_size); + weak_pointers.resize(weak_pointers_size); +} + IQueryTreeNode::IQueryTreeNode(size_t children_size) { children.resize(children_size); } -String IQueryTreeNode::dumpTree() const -{ - WriteBufferFromOwnString buff; - dumpTree(buff); - return buff.str(); -} - -size_t IQueryTreeNode::FormatState::getNodeId(const IQueryTreeNode * node) -{ - auto [it, _] = node_to_id.emplace(node, node_to_id.size()); - return it->second; -} - -void IQueryTreeNode::dumpTree(WriteBuffer & buffer) const -{ - FormatState state; - dumpTreeImpl(buffer, state, 0); -} - bool IQueryTreeNode::isEqual(const IQueryTreeNode & rhs) const { - if (getNodeType() != rhs.getNodeType()) - return false; + std::vector> nodes_to_process; - if (alias != rhs.alias) - return false; + nodes_to_process.emplace_back(this, &rhs); - if (!isEqualImpl(rhs)) - return false; - - size_t children_size = children.size(); - if (children_size != rhs.children.size()) - return false; - - for (size_t i = 0; i < children_size; ++i) + while (!nodes_to_process.empty()) { - const auto & child = children[i]; - const auto & rhs_child = rhs.children[i]; + auto nodes_to_compare = nodes_to_process.back(); + nodes_to_process.pop_back(); - if (!child && !rhs_child) - continue; - else if (child && !rhs_child) + const auto * lhs_node_to_compare = nodes_to_compare.first; + const auto * rhs_node_to_compare = nodes_to_compare.second; + + assert(lhs_node_to_compare); + assert(rhs_node_to_compare); + + if (lhs_node_to_compare->getNodeType() != rhs_node_to_compare->getNodeType() || + lhs_node_to_compare->alias != rhs_node_to_compare->alias || + !lhs_node_to_compare->isEqualImpl(*rhs_node_to_compare)) + { return false; - else if (!child && rhs_child) + } + + const auto & lhs_children = lhs_node_to_compare->children; + const auto & rhs_children = rhs_node_to_compare->children; + + size_t lhs_children_size = lhs_children.size(); + if (lhs_children_size != rhs_children.size()) return false; - if (!child->isEqual(*rhs.children[i])) + for (size_t i = 0; i < lhs_children_size; ++i) + { + const auto & lhs_child = lhs_children[i]; + const auto & rhs_child = rhs_children[i]; + + if (!lhs_child && !rhs_child) + continue; + else if (lhs_child && !rhs_child) + return false; + else if (!lhs_child && rhs_child) + return false; + + nodes_to_process.emplace_back(lhs_child.get(), rhs_child.get()); + } + + const auto & lhs_weak_pointers = lhs_node_to_compare->weak_pointers; + const auto & rhs_weak_pointers = rhs_node_to_compare->weak_pointers; + + size_t lhs_weak_pointers_size = lhs_weak_pointers.size(); + + if (lhs_weak_pointers_size != rhs_weak_pointers.size()) return false; + + for (size_t i = 0; i < lhs_weak_pointers_size; ++i) + { + auto lhs_strong_pointer = lhs_weak_pointers[i].lock(); + auto rhs_strong_pointer = rhs_weak_pointers[i].lock(); + + if (!lhs_strong_pointer && !rhs_strong_pointer) + continue; + else if (lhs_strong_pointer && !rhs_strong_pointer) + return false; + else if (!lhs_strong_pointer && rhs_strong_pointer) + return false; + + nodes_to_process.emplace_back(lhs_strong_pointer.get(), rhs_strong_pointer.get()); + } } return true; @@ -103,7 +129,54 @@ bool IQueryTreeNode::isEqual(const IQueryTreeNode & rhs) const IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const { HashState hash_state; - updateTreeHash(hash_state); + + std::unordered_map node_to_identifier; + + std::vector nodes_to_process; + nodes_to_process.push_back(this); + + while (!nodes_to_process.empty()) + { + const auto * node_to_process = nodes_to_process.back(); + nodes_to_process.pop_back(); + + auto node_identifier_it = node_to_identifier.find(node_to_process); + if (node_identifier_it != node_to_identifier.end()) + { + hash_state.update(node_identifier_it->second); + continue; + } + + node_to_identifier.emplace(node_to_process, node_to_identifier.size()); + + hash_state.update(static_cast(node_to_process->getNodeType())); + if (!node_to_process->alias.empty()) + { + hash_state.update(node_to_process->alias.size()); + hash_state.update(node_to_process->alias); + } + + node_to_process->updateTreeHashImpl(hash_state); + + hash_state.update(node_to_process->children.size()); + + for (const auto & node_to_process_child : node_to_process->children) + { + if (!node_to_process_child) + continue; + + nodes_to_process.push_back(node_to_process_child.get()); + } + + for (const auto & weak_pointer : node_to_process->weak_pointers) + { + auto strong_pointer = weak_pointer.lock(); + if (!strong_pointer) + continue; + + nodes_to_process.push_back(strong_pointer.get()); + } + } Hash result; hash_state.get128(result); @@ -111,61 +184,37 @@ IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const return result; } -void IQueryTreeNode::updateTreeHash(HashState & state) const -{ - state.update(static_cast(getNodeType())); - if (!alias.empty()) - { - state.update(alias.size()); - state.update(alias); - } - - updateTreeHashImpl(state); - - state.update(children.size()); - - for (const auto & child : children) - { - if (!child) - continue; - - child->updateTreeHash(state); - } -} - QueryTreeNodePtr IQueryTreeNode::clone() const { - /** Main motivation for this method is to allow nodes in query tree have weak pointers to other nodes. - * Main use cases is for column node to have weak pointer to its source. - * Source can be lambda, table, subquery and such information is useful for later analysis stages. + /** Clone tree with this node as root. * * Algorithm - * For each node we clone state and also create mapping old_pointer to new pointer. - * For each cloned node we also update node_pointer_to_update_node_pointers array. + * For each node we clone state and also create mapping old pointer to new pointer. + * For each cloned node we update weak pointers array. * - * After that we can update pointer in node_pointer_to_update_node_pointers using old_pointer to new pointer mapping. + * After that we can update pointer in weak pointers array using old pointer to new pointer mapping. */ std::unordered_map old_pointer_to_new_pointer; - QueryTreePointersToUpdate pointers_to_update_after_clone; + std::vector weak_pointers_to_update_after_clone; QueryTreeNodePtr result_cloned_node_place; - std::deque> nodes_to_clone; + std::vector> nodes_to_clone; nodes_to_clone.emplace_back(this, &result_cloned_node_place); while (!nodes_to_clone.empty()) { - const auto [node_to_clone, cloned_node_place] = nodes_to_clone.front(); - nodes_to_clone.pop_front(); + const auto [node_to_clone, place_for_cloned_node] = nodes_to_clone.back(); + nodes_to_clone.pop_back(); auto node_clone = node_to_clone->cloneImpl(); - *cloned_node_place = node_clone; + *place_for_cloned_node = node_clone; node_clone->setAlias(node_to_clone->alias); node_clone->setOriginalAST(node_to_clone->original_ast); node_clone->children = node_to_clone->children; + node_clone->weak_pointers = node_to_clone->weak_pointers; - node_clone->getPointersToUpdateAfterClone(pointers_to_update_after_clone); old_pointer_to_new_pointer.emplace(node_to_clone, node_clone); for (auto & child : node_clone->children) @@ -175,24 +224,34 @@ QueryTreeNodePtr IQueryTreeNode::clone() const nodes_to_clone.emplace_back(child.get(), &child); } + + for (auto & weak_pointer : node_clone->weak_pointers) + { + weak_pointers_to_update_after_clone.push_back(&weak_pointer); + } } - for (auto & [old_pointer, new_pointer] : pointers_to_update_after_clone) + /** Update weak pointers to new pointers if they were changed during clone. + * To do this we check old pointer to new pointer map, if weak pointer + * strong pointer exists as old pointer in map, reinitialize weak pointer with new pointer. + */ + for (auto & weak_pointer_ptr : weak_pointers_to_update_after_clone) { - auto it = old_pointer_to_new_pointer.find(old_pointer); + assert(weak_pointer_ptr); + auto strong_pointer = weak_pointer_ptr->lock(); + auto it = old_pointer_to_new_pointer.find(strong_pointer.get()); - /** If node had weak pointer to some other node and this node is not valid in cloned subtree part do not clone it. - * It will continue to point to previous location and it is expected. - * - * For example: SELECT id as a, a FROM test_table. - * id is resolved as column and test_table is source. - * a is resolved as id and after resolve must be cloned. - * weak pointer to source from a will point to old id location. - */ + /** If node had weak pointer to some other node and this node is not part of cloned subtree do not update weak pointer. + * It will continue to point to previous location and it is expected. + * + * Example: SELECT id FROM test_table; + * During analysis `id` is resolved as column node and `test_table` is column source. + * If we clone `id` column, result column node weak source pointer will point to the same `test_table` column source. + */ if (it == old_pointer_to_new_pointer.end()) continue; - *new_pointer = it->second; + *weak_pointer_ptr = it->second; } return result_cloned_node_place; @@ -221,4 +280,23 @@ String IQueryTreeNode::formatConvertedASTForErrorMessage() const return toAST()->formatForErrorMessage(); } +String IQueryTreeNode::dumpTree() const +{ + WriteBufferFromOwnString buff; + dumpTree(buff); + return buff.str(); +} + +size_t IQueryTreeNode::FormatState::getNodeId(const IQueryTreeNode * node) +{ + auto [it, _] = node_to_id.emplace(node, node_to_id.size()); + return it->second; +} + +void IQueryTreeNode::dumpTree(WriteBuffer & buffer) const +{ + FormatState state; + dumpTreeImpl(buffer, state, 0); +} + } diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index f462de38f2e..63b00085092 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -51,13 +51,22 @@ enum class QueryTreeNodeType /// Convert query tree node type to string const char * toString(QueryTreeNodeType type); -/** Query tree node represent node in query tree. - * This is base class for all query tree nodes. +/** Query tree is semantical representation of query. + * Query tree node represent node in query tree. + * Query tree node is base class for all query tree nodes. + * + * Important property of query tree is that each query tree node can contain weak pointers to other + * query tree nodes. Keeping weak pointer to other query tree nodes can be useful for example for column + * to keep weak pointer to column source, column source can be table, lambda, subquery and preserving of + * such information can significantly simplify query planning. + * + * Another important property of query tree it must be convertible to AST without losing information. */ class IQueryTreeNode; using QueryTreeNodePtr = std::shared_ptr; -using QueryTreeNodeWeakPtr = std::weak_ptr; using QueryTreeNodes = std::vector; +using QueryTreeNodeWeakPtr = std::weak_ptr; +using QueryTreeWeakNodes = std::vector; class IQueryTreeNode : public TypePromotion { @@ -67,6 +76,7 @@ public: /// Get query tree node type virtual QueryTreeNodeType getNodeType() const = 0; + /// Get query tree node type name const char * getNodeTypeName() const { return toString(getNodeType()); @@ -107,17 +117,12 @@ public: return *constant_value; } + /// Returns constant value with type if node has constant value or null otherwise virtual ConstantValuePtr getConstantValueOrNull() const { return {}; } - /// Dump query tree to string - String dumpTree() const; - - /// Dump query tree to buffer - void dumpTree(WriteBuffer & buffer) const; - /** Is tree equal to other tree with node root. * * Aliases of query tree nodes are compared during isEqual call. @@ -135,9 +140,6 @@ public: */ Hash getTreeHash() const; - /// Update tree hash - void updateTreeHash(HashState & state) const; - /// Get a deep copy of the query tree QueryTreeNodePtr clone() const; @@ -208,17 +210,11 @@ public: return formatConvertedASTForErrorMessage(); } - /// Get query tree node children - QueryTreeNodes & getChildren() - { - return children; - } + /// Dump query tree to string + String dumpTree() const; - /// Get query tree node children - const QueryTreeNodes & getChildren() const - { - return children; - } + /// Dump query tree to buffer + void dumpTree(WriteBuffer & buffer) const; class FormatState { @@ -235,59 +231,48 @@ public: */ virtual void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const = 0; - /** Subclass must compare its internal state with rhs node and do not compare its children with rhs node children. - * Caller must compare node and rhs node children. - * - * This method is not protected because if subclass node has weak pointers to other query tree nodes it must use it - * as part of its isEqualImpl method. - */ - virtual bool isEqualImpl(const IQueryTreeNode & rhs) const = 0; + /// Get query tree node children + QueryTreeNodes & getChildren() + { + return children; + } - /** Subclass must update tree hash with its internal state and do not update tree hash for children. - * Caller must update tree hash for node children. - * - * This method is not protected because if subclass node has weak pointers to other query tree nodes it must use it - * as part of its updateTreeHashImpl method. - */ - virtual void updateTreeHashImpl(HashState & hash_state) const = 0; + /// Get query tree node children + const QueryTreeNodes & getChildren() const + { + return children; + } protected: + /** Construct query tree node. + * Resize children to children size. + * Resize weak pointers to weak pointers size. + */ + explicit IQueryTreeNode(size_t children_size, size_t weak_pointers_size); + /// Construct query tree node and resize children to children size explicit IQueryTreeNode(size_t children_size); - /** Subclass node must convert itself to AST. - * Subclass must convert children to AST. + /** Subclass must compare its internal state with rhs node internal state and do not compare children or weak pointers to other + * query tree nodes. */ - virtual ASTPtr toASTImpl() const = 0; + virtual bool isEqualImpl(const IQueryTreeNode & rhs) const = 0; - /** Subclass must clone only it internal state. - * Subclass children will be cloned separately by caller. + /** Subclass must update tree hash with its internal state and do not update tree hash for children or weak pointers to other + * query tree nodes. + */ + virtual void updateTreeHashImpl(HashState & hash_state) const = 0; + + /** Subclass must clone its internal state and do not clone children or weak pointers to other + * query tree nodes. */ virtual QueryTreeNodePtr cloneImpl() const = 0; - /** If node has weak pointers to other tree nodes during clone they will point to other tree nodes. - * Keeping weak pointer to other tree nodes can be useful for example for column to keep weak pointer to column source. - * Source can be table, lambda, subquery and such information is necessary to preserve. - * - * Example: - * SELECT id FROM table; - * id during query analysis will be resolved as ColumnNode and source will be TableNode. - * During clone we must update id ColumnNode source pointer. - * - * Subclass must save old pointer and place of pointer update into pointers_to_update. - * This method will be called on query tree node after clone. - * - * Root of clone process will update pointers as necessary. - */ - using QueryTreePointerToUpdate = std::pair; - using QueryTreePointersToUpdate = std::vector; - - virtual void getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update) - { - (void)(pointers_to_update); - } + /// Subclass must convert its internal state and its children to AST + virtual ASTPtr toASTImpl() const = 0; QueryTreeNodes children; + QueryTreeWeakNodes weak_pointers; private: String alias; diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp index 172a3ef86be..6df3ad0fd51 100644 --- a/src/Analyzer/IdentifierNode.cpp +++ b/src/Analyzer/IdentifierNode.cpp @@ -62,15 +62,15 @@ void IdentifierNode::updateTreeHashImpl(HashState & state) const table_expression_modifiers->updateTreeHash(state); } +QueryTreeNodePtr IdentifierNode::cloneImpl() const +{ + return std::make_shared(identifier); +} + ASTPtr IdentifierNode::toASTImpl() const { auto identifier_parts = identifier.getParts(); return std::make_shared(std::move(identifier_parts)); } -QueryTreeNodePtr IdentifierNode::cloneImpl() const -{ - return std::make_shared(identifier); -} - } diff --git a/src/Analyzer/IdentifierNode.h b/src/Analyzer/IdentifierNode.h index 326dc43c030..7a2351f0ece 100644 --- a/src/Analyzer/IdentifierNode.h +++ b/src/Analyzer/IdentifierNode.h @@ -57,15 +57,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: Identifier identifier; std::optional table_expression_modifiers; diff --git a/src/Analyzer/InterpolateNode.cpp b/src/Analyzer/InterpolateNode.cpp index 3bb211343fa..c6290aa6802 100644 --- a/src/Analyzer/InterpolateNode.cpp +++ b/src/Analyzer/InterpolateNode.cpp @@ -48,6 +48,11 @@ void InterpolateNode::updateTreeHashImpl(HashState &) const /// No state in interpolate column node } +QueryTreeNodePtr InterpolateNode::cloneImpl() const +{ + return std::make_shared(nullptr /*expression*/, nullptr /*interpolate_expression*/); +} + ASTPtr InterpolateNode::toASTImpl() const { auto result = std::make_shared(); @@ -58,9 +63,4 @@ ASTPtr InterpolateNode::toASTImpl() const return result; } -QueryTreeNodePtr InterpolateNode::cloneImpl() const -{ - return std::make_shared(nullptr /*expression*/, nullptr /*interpolate_expression*/); -} - } diff --git a/src/Analyzer/InterpolateNode.h b/src/Analyzer/InterpolateNode.h index 55270594d2f..ba7ef47756c 100644 --- a/src/Analyzer/InterpolateNode.h +++ b/src/Analyzer/InterpolateNode.h @@ -53,15 +53,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & hash_state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: static constexpr size_t expression_child_index = 0; static constexpr size_t interpolate_expression_child_index = 1; diff --git a/src/Analyzer/JoinNode.cpp b/src/Analyzer/JoinNode.cpp index 08213b5622d..315af809e7f 100644 --- a/src/Analyzer/JoinNode.cpp +++ b/src/Analyzer/JoinNode.cpp @@ -106,6 +106,11 @@ ASTPtr JoinNode::toASTTableJoin() const return join_ast; } +QueryTreeNodePtr JoinNode::cloneImpl() const +{ + return std::make_shared(getLeftTableExpression(), getRightTableExpression(), getJoinExpression(), locality, strictness, kind); +} + ASTPtr JoinNode::toASTImpl() const { ASTPtr tables_in_select_query_ast = std::make_shared(); @@ -139,9 +144,4 @@ ASTPtr JoinNode::toASTImpl() const return tables_in_select_query_ast; } -QueryTreeNodePtr JoinNode::cloneImpl() const -{ - return std::make_shared(getLeftTableExpression(), getRightTableExpression(), getJoinExpression(), locality, strictness, kind); -} - } diff --git a/src/Analyzer/JoinNode.h b/src/Analyzer/JoinNode.h index 738fdccd6db..36ba03f6f6c 100644 --- a/src/Analyzer/JoinNode.h +++ b/src/Analyzer/JoinNode.h @@ -129,15 +129,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: JoinLocality locality = JoinLocality::Unspecified; JoinStrictness strictness = JoinStrictness::Unspecified; diff --git a/src/Analyzer/LambdaNode.cpp b/src/Analyzer/LambdaNode.cpp index 8a40072c103..ccc43e75644 100644 --- a/src/Analyzer/LambdaNode.cpp +++ b/src/Analyzer/LambdaNode.cpp @@ -65,6 +65,11 @@ void LambdaNode::updateTreeHashImpl(HashState & state) const } } +QueryTreeNodePtr LambdaNode::cloneImpl() const +{ + return std::make_shared(argument_names, getExpression()); +} + ASTPtr LambdaNode::toASTImpl() const { auto lambda_function_arguments_ast = std::make_shared(); @@ -85,9 +90,4 @@ ASTPtr LambdaNode::toASTImpl() const return lambda_function_ast; } -QueryTreeNodePtr LambdaNode::cloneImpl() const -{ - return std::make_shared(argument_names, getExpression()); -} - } diff --git a/src/Analyzer/LambdaNode.h b/src/Analyzer/LambdaNode.h index 3d4ff1c6c55..3b41cdc9c78 100644 --- a/src/Analyzer/LambdaNode.h +++ b/src/Analyzer/LambdaNode.h @@ -102,15 +102,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: Names argument_names; diff --git a/src/Analyzer/ListNode.cpp b/src/Analyzer/ListNode.cpp index f9affa133cf..b98e29deecd 100644 --- a/src/Analyzer/ListNode.cpp +++ b/src/Analyzer/ListNode.cpp @@ -67,6 +67,11 @@ void ListNode::updateTreeHashImpl(HashState &) const /// No state } +QueryTreeNodePtr ListNode::cloneImpl() const +{ + return std::make_shared(); +} + ASTPtr ListNode::toASTImpl() const { auto expression_list_ast = std::make_shared(); @@ -80,9 +85,4 @@ ASTPtr ListNode::toASTImpl() const return expression_list_ast; } -QueryTreeNodePtr ListNode::cloneImpl() const -{ - return std::make_shared(); -} - } diff --git a/src/Analyzer/ListNode.h b/src/Analyzer/ListNode.h index 0f44b500057..7d941731b25 100644 --- a/src/Analyzer/ListNode.h +++ b/src/Analyzer/ListNode.h @@ -43,14 +43,14 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState &) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + + ASTPtr toASTImpl() const override; }; } diff --git a/src/Analyzer/MatcherNode.cpp b/src/Analyzer/MatcherNode.cpp index 21a80b1608b..95390455826 100644 --- a/src/Analyzer/MatcherNode.cpp +++ b/src/Analyzer/MatcherNode.cpp @@ -246,6 +246,19 @@ void MatcherNode::updateTreeHashImpl(HashState & hash_state) const } } +QueryTreeNodePtr MatcherNode::cloneImpl() const +{ + MatcherNodePtr matcher_node = std::make_shared(); + + matcher_node->matcher_type = matcher_type; + matcher_node->qualified_identifier = qualified_identifier; + matcher_node->columns_identifiers = columns_identifiers; + matcher_node->columns_matcher = columns_matcher; + matcher_node->columns_identifiers_set = columns_identifiers_set; + + return matcher_node; +} + ASTPtr MatcherNode::toASTImpl() const { ASTPtr result; @@ -292,17 +305,4 @@ ASTPtr MatcherNode::toASTImpl() const return result; } -QueryTreeNodePtr MatcherNode::cloneImpl() const -{ - MatcherNodePtr matcher_node = std::make_shared(); - - matcher_node->matcher_type = matcher_type; - matcher_node->qualified_identifier = qualified_identifier; - matcher_node->columns_identifiers = columns_identifiers; - matcher_node->columns_matcher = columns_matcher; - matcher_node->columns_identifiers_set = columns_identifiers_set; - - return matcher_node; -} - } diff --git a/src/Analyzer/MatcherNode.h b/src/Analyzer/MatcherNode.h index b489f98588a..f989ac9bb7e 100644 --- a/src/Analyzer/MatcherNode.h +++ b/src/Analyzer/MatcherNode.h @@ -151,15 +151,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & hash_state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: explicit MatcherNode(MatcherNodeType matcher_type_, Identifier qualified_identifier_, diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 6acc70d898b..23457256979 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -338,6 +338,26 @@ void QueryNode::updateTreeHashImpl(HashState & state) const table_expression_modifiers->updateTreeHash(state); } +QueryTreeNodePtr QueryNode::cloneImpl() const +{ + auto result_query_node = std::make_shared(); + + result_query_node->is_subquery = is_subquery; + result_query_node->is_cte = is_cte; + result_query_node->is_distinct = is_distinct; + result_query_node->is_limit_with_ties = is_limit_with_ties; + result_query_node->is_group_by_with_totals = is_group_by_with_totals; + result_query_node->is_group_by_with_rollup = is_group_by_with_rollup; + result_query_node->is_group_by_with_cube = is_group_by_with_cube; + result_query_node->is_group_by_with_grouping_sets = is_group_by_with_grouping_sets; + result_query_node->cte_name = cte_name; + result_query_node->projection_columns = projection_columns; + result_query_node->constant_value = constant_value; + result_query_node->table_expression_modifiers = table_expression_modifiers; + + return result_query_node; +} + ASTPtr QueryNode::toASTImpl() const { auto select_query = std::make_shared(); @@ -415,24 +435,4 @@ ASTPtr QueryNode::toASTImpl() const return result_select_query; } -QueryTreeNodePtr QueryNode::cloneImpl() const -{ - auto result_query_node = std::make_shared(); - - result_query_node->is_subquery = is_subquery; - result_query_node->is_cte = is_cte; - result_query_node->is_distinct = is_distinct; - result_query_node->is_limit_with_ties = is_limit_with_ties; - result_query_node->is_group_by_with_totals = is_group_by_with_totals; - result_query_node->is_group_by_with_rollup = is_group_by_with_rollup; - result_query_node->is_group_by_with_cube = is_group_by_with_cube; - result_query_node->is_group_by_with_grouping_sets = is_group_by_with_grouping_sets; - result_query_node->cte_name = cte_name; - result_query_node->projection_columns = projection_columns; - result_query_node->constant_value = constant_value; - result_query_node->table_expression_modifiers = table_expression_modifiers; - - return result_query_node; -} - } diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 821df4d666e..d82cb0f903f 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -447,15 +447,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState &) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: bool is_subquery = false; bool is_cte = false; diff --git a/src/Analyzer/SortNode.cpp b/src/Analyzer/SortNode.cpp index 5bdb9b9adf7..509a973c7d4 100644 --- a/src/Analyzer/SortNode.cpp +++ b/src/Analyzer/SortNode.cpp @@ -135,6 +135,11 @@ void SortNode::updateTreeHashImpl(HashState & hash_state) const } } +QueryTreeNodePtr SortNode::cloneImpl() const +{ + return std::make_shared(nullptr /*expression*/, sort_direction, nulls_sort_direction, collator, with_fill); +} + ASTPtr SortNode::toASTImpl() const { auto result = std::make_shared(); @@ -156,9 +161,4 @@ ASTPtr SortNode::toASTImpl() const return result; } -QueryTreeNodePtr SortNode::cloneImpl() const -{ - return std::make_shared(nullptr /*expression*/, sort_direction, nulls_sort_direction, collator, with_fill); -} - } diff --git a/src/Analyzer/SortNode.h b/src/Analyzer/SortNode.h index f003cd34e00..a3c722db698 100644 --- a/src/Analyzer/SortNode.h +++ b/src/Analyzer/SortNode.h @@ -129,15 +129,16 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & hash_state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: static constexpr size_t sort_expression_child_index = 0; static constexpr size_t fill_from_child_index = 1; diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index b0ade7ca1c4..2056e10ff6f 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -120,6 +120,18 @@ void TableFunctionNode::updateTreeHashImpl(HashState & state) const table_expression_modifiers->updateTreeHash(state); } +QueryTreeNodePtr TableFunctionNode::cloneImpl() const +{ + auto result = std::make_shared(table_function_name); + + result->storage = storage; + result->storage_id = storage_id; + result->storage_snapshot = storage_snapshot; + result->table_expression_modifiers = table_expression_modifiers; + + return result; +} + ASTPtr TableFunctionNode::toASTImpl() const { auto table_function_ast = std::make_shared(); @@ -133,16 +145,4 @@ ASTPtr TableFunctionNode::toASTImpl() const return table_function_ast; } -QueryTreeNodePtr TableFunctionNode::cloneImpl() const -{ - auto result = std::make_shared(table_function_name); - - result->storage = storage; - result->storage_id = storage_id; - result->storage_snapshot = storage_snapshot; - result->table_expression_modifiers = table_expression_modifiers; - - return result; -} - } diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index 979a930542b..6dc6548cd1a 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -131,15 +131,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: String table_function_name; TableFunctionPtr table_function; diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index fc25d192f3a..3075bce238a 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -71,11 +71,6 @@ String TableNode::getName() const return storage->getStorageID().getFullNameNotQuoted(); } -ASTPtr TableNode::toASTImpl() const -{ - return std::make_shared(storage_id.getDatabaseName(), storage_id.getTableName()); -} - QueryTreeNodePtr TableNode::cloneImpl() const { auto result_table_node = std::make_shared(storage, storage_id, storage_lock, storage_snapshot); @@ -84,4 +79,9 @@ QueryTreeNodePtr TableNode::cloneImpl() const return result_table_node; } +ASTPtr TableNode::toASTImpl() const +{ + return std::make_shared(storage_id.getDatabaseName(), storage_id.getTableName()); +} + } diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index 216dc516246..e154f62614a 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -84,15 +84,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: StoragePtr storage; StorageID storage_id; diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index b3fa07a828f..4cbda0f57cf 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -211,6 +211,22 @@ void UnionNode::updateTreeHashImpl(HashState & state) const table_expression_modifiers->updateTreeHash(state); } +QueryTreeNodePtr UnionNode::cloneImpl() const +{ + auto result_query_node = std::make_shared(); + + result_query_node->is_subquery = is_subquery; + result_query_node->is_cte = is_cte; + result_query_node->cte_name = cte_name; + result_query_node->union_mode = union_mode; + result_query_node->union_modes = union_modes; + result_query_node->union_modes_set = union_modes_set; + result_query_node->constant_value = constant_value; + result_query_node->table_expression_modifiers = table_expression_modifiers; + + return result_query_node; +} + ASTPtr UnionNode::toASTImpl() const { auto select_with_union_query = std::make_shared(); @@ -229,20 +245,4 @@ ASTPtr UnionNode::toASTImpl() const return select_with_union_query; } -QueryTreeNodePtr UnionNode::cloneImpl() const -{ - auto result_query_node = std::make_shared(); - - result_query_node->is_subquery = is_subquery; - result_query_node->is_cte = is_cte; - result_query_node->cte_name = cte_name; - result_query_node->union_mode = union_mode; - result_query_node->union_modes = union_modes; - result_query_node->union_modes_set = union_modes_set; - result_query_node->constant_value = constant_value; - result_query_node->table_expression_modifiers = table_expression_modifiers; - - return result_query_node; -} - } diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h index 7ddc0b5381c..43d5f6bd345 100644 --- a/src/Analyzer/UnionNode.h +++ b/src/Analyzer/UnionNode.h @@ -149,15 +149,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState &) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: bool is_subquery = false; bool is_cte = false; diff --git a/src/Analyzer/WindowNode.cpp b/src/Analyzer/WindowNode.cpp index 0378c8746ed..4b92054b103 100644 --- a/src/Analyzer/WindowNode.cpp +++ b/src/Analyzer/WindowNode.cpp @@ -166,6 +166,14 @@ void WindowNode::updateTreeHashImpl(HashState & hash_state) const hash_state.update(window_frame.end_preceding); } +QueryTreeNodePtr WindowNode::cloneImpl() const +{ + auto window_node = std::make_shared(window_frame); + window_node->parent_window_name = parent_window_name; + + return window_node; +} + ASTPtr WindowNode::toASTImpl() const { auto window_definition = std::make_shared(); @@ -198,12 +206,4 @@ ASTPtr WindowNode::toASTImpl() const return window_definition; } -QueryTreeNodePtr WindowNode::cloneImpl() const -{ - auto window_node = std::make_shared(window_frame); - window_node->parent_window_name = parent_window_name; - - return window_node; -} - } diff --git a/src/Analyzer/WindowNode.h b/src/Analyzer/WindowNode.h index e99b66088ca..57875aad710 100644 --- a/src/Analyzer/WindowNode.h +++ b/src/Analyzer/WindowNode.h @@ -170,15 +170,15 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; +protected: bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & hash_state) const override; -protected: - ASTPtr toASTImpl() const override; - QueryTreeNodePtr cloneImpl() const override; + ASTPtr toASTImpl() const override; + private: static constexpr size_t order_by_child_index = 0; static constexpr size_t partition_by_child_index = 1; diff --git a/src/Analyzer/tests/gtest_query_tree_node.cpp b/src/Analyzer/tests/gtest_query_tree_node.cpp index f29a0f115d7..079869b2a53 100644 --- a/src/Analyzer/tests/gtest_query_tree_node.cpp +++ b/src/Analyzer/tests/gtest_query_tree_node.cpp @@ -31,15 +31,15 @@ public: { } - ASTPtr toASTImpl() const override - { - return nullptr; - } - QueryTreeNodePtr cloneImpl() const override { return std::make_shared(); } + + ASTPtr toASTImpl() const override + { + return nullptr; + } }; TEST(QueryTreeNode, Clone) From 77ea9a11cd349498290bf182eb47a4ee67dd8cf0 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 11 Oct 2022 15:19:50 +0200 Subject: [PATCH 129/188] Fixed code review issues --- ...FunctionNodes.cpp => AggregationUtils.cpp} | 39 +- ...gateFunctionNodes.h => AggregationUtils.h} | 5 + src/Analyzer/Identifier.h | 49 + src/Analyzer/MatcherNode.cpp | 10 +- src/Analyzer/QueryAnalysisPass.cpp | 959 +++++++++--------- src/Analyzer/QueryTreePassManager.cpp | 2 - src/Analyzer/Utils.cpp | 2 +- ...tionNodes.cpp => WindowFunctionsUtils.cpp} | 2 +- ...FunctionNodes.h => WindowFunctionsUtils.h} | 0 src/Planner/Planner.cpp | 4 +- src/Planner/PlannerAggregation.cpp | 2 +- src/Planner/PlannerExpressionAnalysis.cpp | 4 +- ...yzer_aliases_compound_expression.reference | 7 + ...5_analyzer_aliases_compound_expression.sql | 21 + 14 files changed, 615 insertions(+), 491 deletions(-) rename src/Analyzer/{CollectAggregateFunctionNodes.cpp => AggregationUtils.cpp} (61%) rename src/Analyzer/{CollectAggregateFunctionNodes.h => AggregationUtils.h} (75%) rename src/Analyzer/{CollectWindowFunctionNodes.cpp => WindowFunctionsUtils.cpp} (97%) rename src/Analyzer/{CollectWindowFunctionNodes.h => WindowFunctionsUtils.h} (100%) create mode 100644 tests/queries/0_stateless/02385_analyzer_aliases_compound_expression.reference create mode 100644 tests/queries/0_stateless/02385_analyzer_aliases_compound_expression.sql diff --git a/src/Analyzer/CollectAggregateFunctionNodes.cpp b/src/Analyzer/AggregationUtils.cpp similarity index 61% rename from src/Analyzer/CollectAggregateFunctionNodes.cpp rename to src/Analyzer/AggregationUtils.cpp index 8629fa09902..47ec15c647d 100644 --- a/src/Analyzer/CollectAggregateFunctionNodes.cpp +++ b/src/Analyzer/AggregationUtils.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -74,4 +74,41 @@ void assertNoAggregateFunctionNodes(const QueryTreeNodePtr & node, const String visitor.visit(node); } +namespace +{ + +class ValidateGroupingFunctionNodesVisitor : public ConstInDepthQueryTreeVisitor +{ +public: + explicit ValidateGroupingFunctionNodesVisitor(String assert_no_grouping_function_place_message_) + : assert_no_grouping_function_place_message(std::move(assert_no_grouping_function_place_message_)) + {} + + void visitImpl(const QueryTreeNodePtr & node) + { + auto * function_node = node->as(); + if (function_node && function_node->getFunctionName() == "grouping") + throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, + "GROUPING function {} is found {} in query", + function_node->formatASTForErrorMessage(), + assert_no_grouping_function_place_message); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) + { + return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION); + } + +private: + String assert_no_grouping_function_place_message; +}; + +} + +void assertNoGroupingFunction(const QueryTreeNodePtr & node, const String & assert_no_grouping_function_place_message) +{ + ValidateGroupingFunctionNodesVisitor visitor(assert_no_grouping_function_place_message); + visitor.visit(node); +} + } diff --git a/src/Analyzer/CollectAggregateFunctionNodes.h b/src/Analyzer/AggregationUtils.h similarity index 75% rename from src/Analyzer/CollectAggregateFunctionNodes.h rename to src/Analyzer/AggregationUtils.h index 5e68ef67d8b..c2e53e55c04 100644 --- a/src/Analyzer/CollectAggregateFunctionNodes.h +++ b/src/Analyzer/AggregationUtils.h @@ -20,4 +20,9 @@ void collectAggregateFunctionNodes(const QueryTreeNodePtr & node, QueryTreeNodes */ void assertNoAggregateFunctionNodes(const QueryTreeNodePtr & node, const String & assert_no_aggregates_place_message); +/** Assert that there are no GROUPING functions in node children. + * Do not visit subqueries. + */ +void assertNoGroupingFunction(const QueryTreeNodePtr & node, const String & assert_no_grouping_function_place_message); + } diff --git a/src/Analyzer/Identifier.h b/src/Analyzer/Identifier.h index 9d70d4dfc41..2252ce2854f 100644 --- a/src/Analyzer/Identifier.h +++ b/src/Analyzer/Identifier.h @@ -3,6 +3,9 @@ #include #include +#include +#include + #include #include @@ -361,3 +364,49 @@ inline std::ostream & operator<<(std::ostream & stream, const IdentifierView & i } } + +/// See https://fmt.dev/latest/api.html#formatting-user-defined-types + +template <> +struct fmt::formatter +{ + constexpr static auto parse(format_parse_context & ctx) + { + const auto * it = ctx.begin(); + const auto * end = ctx.end(); + + /// Only support {}. + if (it != end && *it != '}') + throw format_error("invalid format"); + + return it; + } + + template + auto format(const DB::Identifier & identifier, FormatContext & ctx) + { + return format_to(ctx.out(), "{}", identifier.getFullName()); + } +}; + +template <> +struct fmt::formatter +{ + constexpr static auto parse(format_parse_context & ctx) + { + const auto * it = ctx.begin(); + const auto * end = ctx.end(); + + /// Only support {}. + if (it != end && *it != '}') + throw format_error("invalid format"); + + return it; + } + + template + auto format(const DB::IdentifierView & identifier_view, FormatContext & ctx) + { + return format_to(ctx.out(), "{}", identifier_view.getFullName()); + } +}; diff --git a/src/Analyzer/MatcherNode.cpp b/src/Analyzer/MatcherNode.cpp index 95390455826..16633be13df 100644 --- a/src/Analyzer/MatcherNode.cpp +++ b/src/Analyzer/MatcherNode.cpp @@ -134,15 +134,7 @@ void MatcherNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, } else if (matcher_type == MatcherNodeType::COLUMNS_LIST) { - buffer << ", columns_identifiers: "; - size_t columns_identifiers_size = columns_identifiers.size(); - for (size_t i = 0; i < columns_identifiers_size; ++i) - { - buffer << columns_identifiers[i].getFullName(); - - if (i + 1 != columns_identifiers_size) - buffer << ", "; - } + buffer << ", " << fmt::format("column_identifiers: {}", fmt::join(columns_identifiers, ", ")); } const auto & column_transformers_list = getColumnTransformers(); diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index 04d636c5c58..cc62782c0e2 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -1,21 +1,8 @@ #include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include #include #include @@ -33,6 +20,8 @@ #include #include +#include +#include #include #include #include @@ -51,8 +40,6 @@ #include #include #include -#include -#include #include @@ -64,12 +51,13 @@ #include #include #include -#include #include #include #include #include +#include + #include #include @@ -99,7 +87,6 @@ namespace ErrorCodes extern const int TOO_DEEP_SUBQUERIES; extern const int UNKNOWN_AGGREGATE_FUNCTION; extern const int NOT_AN_AGGREGATE; - extern const int ILLEGAL_AGGREGATION; extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_FINAL; @@ -195,6 +182,7 @@ namespace ErrorCodes * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. * TODO: Support function identifier resolve from parent query scope, if lambda in parent scope does not capture any columns. * TODO: Support group_by_use_nulls + * TODO: Scalar subqueries cache. */ namespace @@ -371,25 +359,51 @@ struct IdentifierResolveSettings /// Allow to check database catalog during table identifier resolution bool allow_to_check_database_catalog = true; + + /// Allow to resolve subquery during identifier resolution + bool allow_to_resolve_subquery_during_identifier_resolution = true; }; +struct StringTransparentHash +{ + using is_transparent = void; + using hash = std::hash; + + [[maybe_unused]] size_t operator()(const char * data) const + { + return hash()(data); + } + + size_t operator()(std::string_view data) const + { + return hash()(data); + } + + size_t operator()(const std::string & data) const + { + return hash()(data); + } +}; + +using ColumnNameToColumnNodeMap = std::unordered_map>; + struct TableExpressionData { std::string table_expression_name; std::string table_expression_description; std::string table_name; std::string database_name; - std::unordered_map column_name_to_column_node; - std::unordered_set column_identifier_first_parts; + ColumnNameToColumnNodeMap column_name_to_column_node; + std::unordered_set> column_identifier_first_parts; bool hasFullIdentifierName(IdentifierView identifier) const { - return column_name_to_column_node.contains(std::string(identifier.getFullName())); + return column_name_to_column_node.contains(std::string_view(identifier.getFullName())); } bool canBindIdentifier(IdentifierView identifier) const { - return column_identifier_first_parts.contains(std::string(identifier.at(0))); + return column_identifier_first_parts.contains(std::string_view(identifier.at(0))); } [[maybe_unused]] void dump(WriteBuffer & buffer) const @@ -643,7 +657,7 @@ struct IdentifierResolveScope std::unordered_map alias_name_to_table_expression_node; /// Table column name to column node. Valid only during table ALIAS columns resolve. - std::unordered_map column_name_to_column_node; + ColumnNameToColumnNodeMap column_name_to_column_node; /// CTE name to query node std::unordered_map cte_name_to_query_node; @@ -953,52 +967,69 @@ public: auto node_type = node->getNodeType(); - if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) + switch (node_type) { - if (table_expression) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "For query or union analysis table expression must be empty"); + case QueryTreeNodeType::QUERY: + { + if (table_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "For query analysis table expression must be empty"); - if (node_type == QueryTreeNodeType::QUERY) resolveQuery(node, scope); - else + break; + } + case QueryTreeNodeType::UNION: + { + if (table_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "For union analysis table expression must be empty"); + resolveUnion(node, scope); - } - else if (node_type == QueryTreeNodeType::CONSTANT || node_type == QueryTreeNodeType::IDENTIFIER || node_type == QueryTreeNodeType::COLUMN || - node_type == QueryTreeNodeType::FUNCTION || node_type == QueryTreeNodeType::LIST) - { - if (!table_expression) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "For expression analysis table expression must not be empty"); + break; + } + case QueryTreeNodeType::IDENTIFIER: + [[fallthrough]]; + case QueryTreeNodeType::CONSTANT: + [[fallthrough]]; + case QueryTreeNodeType::COLUMN: + [[fallthrough]]; + case QueryTreeNodeType::FUNCTION: + [[fallthrough]]; + case QueryTreeNodeType::LIST: + { + if (!table_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "For expression analysis table expression must not be empty"); - scope.expression_join_tree_node = table_expression; - validateTableExpressionModifiers(scope.expression_join_tree_node, scope); - initializeTableExpressionColumns(scope.expression_join_tree_node, scope); + scope.expression_join_tree_node = table_expression; + validateTableExpressionModifiers(scope.expression_join_tree_node, scope); + initializeTableExpressionColumns(scope.expression_join_tree_node, scope); - if (node_type == QueryTreeNodeType::LIST) - resolveExpressionNodeList(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - else - resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Node {} with type {} is not supported by query analyzer. Supported nodes are union, query, constant, identifier, column, function, list.", - node->formatASTForErrorMessage(), - node->getNodeTypeName()); + if (node_type == QueryTreeNodeType::LIST) + resolveExpressionNodeList(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + else + resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + break; + } + default: + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Node {} with type {} is not supported by query analyzer. Supported nodes are query, union, identifier, constant, column, function, list.", + node->formatASTForErrorMessage(), + node->getNodeTypeName()); + } } } private: /// Utility functions - static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path); + static bool isExpressionNodeType(QueryTreeNodeType node_type); - QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); + static bool isFunctionExpressionNodeType(QueryTreeNodeType node_type); - void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, size_t subquery_depth); - - void validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + static bool isTableExpressionNodeType(QueryTreeNodeType node_type); static ProjectionName calculateFunctionProjectionName(const QueryTreeNodePtr & function_node, const ProjectionNames & parameters_projection_names, @@ -1018,14 +1049,22 @@ private: const ProjectionName & fill_to_expression_projection_name, const ProjectionName & fill_step_expression_projection_name); - static void validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope); + static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path); - static void validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); + + void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, size_t subquery_depth); static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope); static void replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope); + static void validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope); + + static void validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + + void validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + /// Resolve identifier functions QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier); @@ -1118,177 +1157,22 @@ private: /// Utility functions implementation -/** Wrap expression node in tuple element function calls for nested paths. - * Example: Expression node: compound_expression. Nested path: nested_path_1.nested_path_2. - * Result: tupleElement(tupleElement(compound_expression, 'nested_path_1'), 'nested_path_2'). - */ -QueryTreeNodePtr QueryAnalyzer::wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path) -{ - size_t nested_path_parts_size = nested_path.getPartsSize(); - for (size_t i = 0; i < nested_path_parts_size; ++i) - { - const auto & nested_path_part = nested_path[i]; - auto tuple_element_function = std::make_shared("tupleElement"); - tuple_element_function->getArguments().getNodes().push_back(expression_node); - tuple_element_function->getArguments().getNodes().push_back(std::make_shared(nested_path_part)); - expression_node = tuple_element_function; - } - return expression_node; +bool QueryAnalyzer::isExpressionNodeType(QueryTreeNodeType node_type) +{ + return node_type == QueryTreeNodeType::CONSTANT || node_type == QueryTreeNodeType::COLUMN || node_type == QueryTreeNodeType::FUNCTION + || node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION; } -/** Try to get lambda node from sql user defined functions if sql user defined function with function name exists. - * Returns lambda node if function exists, nullptr otherwise. - */ -QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name) +bool QueryAnalyzer::isFunctionExpressionNodeType(QueryTreeNodeType node_type) { - auto user_defined_function = UserDefinedSQLFunctionFactory::instance().tryGet(function_name); - if (!user_defined_function) - return {}; - - const auto & create_function_query = user_defined_function->as(); - auto result_node = buildQueryTree(create_function_query->function_core, context); - if (result_node->getNodeType() != QueryTreeNodeType::LAMBDA) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "SQL user defined function {} must represent lambda expression. Actual {}", - function_name, - create_function_query->function_core->formatForErrorMessage()); - - return result_node; + return node_type == QueryTreeNodeType::LAMBDA; } -/// Evaluate scalar subquery and perform constant folding. -void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size_t subquery_depth) +bool QueryAnalyzer::isTableExpressionNodeType(QueryTreeNodeType node_type) { - auto * query_node = node->as(); - auto * union_node = node->as(); - if (!query_node && !union_node) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Node must have query or union type. Actual {} {}", - node->getNodeTypeName(), - node->formatASTForErrorMessage()); - - if ((query_node && query_node->hasConstantValue()) || - (union_node && union_node->hasConstantValue())) - return; - - auto subquery_context = Context::createCopy(context); - Settings subquery_settings = context->getSettings(); - subquery_settings.max_result_rows = 1; - subquery_settings.extremes = false; - subquery_context->setSettings(subquery_settings); - - if (!subquery_context->hasQueryContext()) - { - // auto subquery_query_context = subquery_context->getQueryContext(); - // for (const auto & it : data.scalars) - // context->addScalar(it.first, it.second); - } - - auto options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth, true /*is_subquery*/); - auto interpreter = std::make_unique(node, options, subquery_context); - - auto io = interpreter->execute(); - - Block block; - PullingAsyncPipelineExecutor executor(io.pipeline); - io.pipeline.setProgressCallback(context->getProgressCallback()); - while (block.rows() == 0 && executor.pull(block)) - { - } - - if (block.rows() == 0) - { - auto types = interpreter->getSampleBlock().getDataTypes(); - if (types.size() != 1) - types = {std::make_shared(types)}; - - auto & type = types[0]; - if (!type->isNullable()) - { - if (!type->canBeInsideNullable()) - throw Exception(ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY, - "Scalar subquery returned empty result of type {} which cannot be Nullable.", - type->getName()); - - type = makeNullable(type); - } - - auto constant_value = std::make_shared(Null(), std::move(type)); - - if (query_node) - query_node->performConstantFolding(std::move(constant_value)); - else if (union_node) - query_node->performConstantFolding(std::move(constant_value)); - - return; - } - - if (block.rows() != 1) - throw Exception(ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY, "Scalar subquery returned more than one row"); - - Block tmp_block; - while (tmp_block.rows() == 0 && executor.pull(tmp_block)) - { - } - - if (tmp_block.rows() != 0) - throw Exception(ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY, "Scalar subquery returned more than one row"); - - block = materializeBlock(block); - size_t columns = block.columns(); - - // Block scalar; - Field scalar_value; - DataTypePtr scalar_type; - - if (columns == 1) - { - auto & column = block.getByPosition(0); - /// Here we wrap type to nullable if we can. - /// It is needed cause if subquery return no rows, it's result will be Null. - /// In case of many columns, do not check it cause tuple can't be nullable. - if (!column.type->isNullable() && column.type->canBeInsideNullable()) - { - column.type = makeNullable(column.type); - column.column = makeNullable(column.column); - } - - column.column->get(0, scalar_value); - scalar_type = column.type; - } - else - { - auto tuple_column = ColumnTuple::create(block.getColumns()); - tuple_column->get(0, scalar_value); - scalar_type = std::make_shared(block.getDataTypes(), block.getNames()); - } - - auto constant_value = std::make_shared(std::move(scalar_value), std::move(scalar_type)); - if (query_node) - query_node->performConstantFolding(std::move(constant_value)); - else if (union_node) - union_node->performConstantFolding(std::move(constant_value)); -} - -void QueryAnalyzer::validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) -{ - if (!context->getSettingsRef().joined_subquery_requires_alias) - return; - - bool table_expression_has_alias = table_expression_node->hasAlias(); - if (table_expression_has_alias) - return; - - auto table_expression_node_type = table_expression_node->getNodeType(); - if (table_expression_node_type == QueryTreeNodeType::TABLE_FUNCTION || - table_expression_node_type == QueryTreeNodeType::QUERY || - table_expression_node_type == QueryTreeNodeType::UNION) - throw Exception(ErrorCodes::ALIAS_REQUIRED, - "JOIN {} no alias for subquery or table function {}. In scope {} (set joined_subquery_requires_alias = 0 to disable restriction)", - join_node->formatASTForErrorMessage(), - table_expression_node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); + return node_type == QueryTreeNodeType::TABLE || node_type == QueryTreeNodeType::TABLE_FUNCTION || + node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION; } ProjectionName QueryAnalyzer::calculateFunctionProjectionName(const QueryTreeNodePtr & function_node, const ProjectionNames & parameters_projection_names, @@ -1467,81 +1351,154 @@ ProjectionName QueryAnalyzer::calculateSortColumnProjectionName(const QueryTreeN return sort_column_projection_name_buffer.str(); } -void QueryAnalyzer::validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope) +/** Wrap expression node in tuple element function calls for nested paths. + * Example: Expression node: compound_expression. Nested path: nested_path_1.nested_path_2. + * Result: tupleElement(tupleElement(compound_expression, 'nested_path_1'), 'nested_path_2'). + */ +QueryTreeNodePtr QueryAnalyzer::wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path) { - const auto limit_offset_constant_value = expression_node->getConstantValueOrNull(); - if (!limit_offset_constant_value || !isNativeNumber(removeNullable(limit_offset_constant_value->getType()))) - throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, - "{} expression must be constant with numeric type. Actual {}. In scope {}", - expression_description, - expression_node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); + size_t nested_path_parts_size = nested_path.getPartsSize(); + for (size_t i = 0; i < nested_path_parts_size; ++i) + { + const auto & nested_path_part = nested_path[i]; + auto tuple_element_function = std::make_shared("tupleElement"); - Field converted = convertFieldToType(limit_offset_constant_value->getValue(), DataTypeUInt64()); - if (converted.isNull()) - throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, - "{} numeric constant expression is not representable as UInt64", - expression_description); + auto & tuple_element_function_arguments_nodes = tuple_element_function->getArguments().getNodes(); + tuple_element_function_arguments_nodes.reserve(2); + tuple_element_function_arguments_nodes.push_back(expression_node); + tuple_element_function_arguments_nodes.push_back(std::make_shared(nested_path_part)); + + expression_node = std::move(tuple_element_function); + } + + return expression_node; } -void QueryAnalyzer::validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) +/** Try to get lambda node from sql user defined functions if sql user defined function with function name exists. + * Returns lambda node if function exists, nullptr otherwise. + */ +QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name) { - auto * table_node = table_expression_node->as(); - auto * table_function_node = table_expression_node->as(); - auto * query_node = table_expression_node->as(); - auto * union_node = table_expression_node->as(); + auto user_defined_function = UserDefinedSQLFunctionFactory::instance().tryGet(function_name); + if (!user_defined_function) + return {}; - if (!table_node && !table_function_node && !query_node && !union_node) + const auto & create_function_query = user_defined_function->as(); + auto result_node = buildQueryTree(create_function_query->function_core, context); + if (result_node->getNodeType() != QueryTreeNodeType::LAMBDA) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected table expression. Expected table, table function, query or union node. Actual {}", - table_expression_node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); + "SQL user defined function {} must represent lambda expression. Actual {}", + function_name, + create_function_query->function_core->formatForErrorMessage()); - if (query_node || union_node) + return result_node; +} + +/// Evaluate scalar subquery and perform constant folding if scalar subquery does not have constant value +void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size_t subquery_depth) +{ + auto * query_node = node->as(); + auto * union_node = node->as(); + if (!query_node && !union_node) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Node must have query or union type. Actual {} {}", + node->getNodeTypeName(), + node->formatASTForErrorMessage()); + + if (node->hasConstantValue()) + return; + + auto subquery_context = Context::createCopy(context); + Settings subquery_settings = context->getSettings(); + subquery_settings.max_result_rows = 1; + subquery_settings.extremes = false; + subquery_context->setSettings(subquery_settings); + + auto options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth, true /*is_subquery*/); + auto interpreter = std::make_unique(node, options, subquery_context); + + auto io = interpreter->execute(); + + Block block; + PullingAsyncPipelineExecutor executor(io.pipeline); + io.pipeline.setProgressCallback(context->getProgressCallback()); + + while (block.rows() == 0 && executor.pull(block)) { - auto table_expression_modifiers = query_node ? query_node->getTableExpressionModifiers() : union_node->getTableExpressionModifiers(); - - if (table_expression_modifiers.has_value()) - { - String table_expression_modifiers_error_message; - - if (table_expression_modifiers->hasFinal()) - { - table_expression_modifiers_error_message += "FINAL"; - - if (table_expression_modifiers->hasSampleSizeRatio()) - table_expression_modifiers_error_message += ", SAMPLE"; - } - else if (table_expression_modifiers->hasSampleSizeRatio()) - { - table_expression_modifiers_error_message += "SAMPLE"; - } - - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Table expression modifiers {} are not supported for subquery {}. In scope {}", - table_expression_modifiers_error_message, - table_expression_node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - } } - else if (table_node || table_function_node) + + if (block.rows() == 0) { - auto table_expression_modifiers = table_node ? table_node->getTableExpressionModifiers() : table_function_node->getTableExpressionModifiers(); + auto types = interpreter->getSampleBlock().getDataTypes(); + if (types.size() != 1) + types = {std::make_shared(types)}; - if (table_expression_modifiers.has_value()) + auto & type = types[0]; + if (!type->isNullable()) { - const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); - if (table_expression_modifiers->hasFinal() && !storage->supportsFinal()) - throw Exception(ErrorCodes::ILLEGAL_FINAL, - "Storage {} doesn't support FINAL", - storage->getName()); + if (!type->canBeInsideNullable()) + throw Exception(ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY, + "Scalar subquery returned empty result of type {} which cannot be Nullable.", + type->getName()); - if (table_expression_modifiers->hasSampleSizeRatio() && !storage->supportsSampling()) - throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, - "Storage {} doesn't support sampling", - storage->getStorageID().getFullNameNotQuoted()); + type = makeNullable(type); } + + auto constant_value = std::make_shared(Null(), std::move(type)); + + if (query_node) + query_node->performConstantFolding(std::move(constant_value)); + else if (union_node) + query_node->performConstantFolding(std::move(constant_value)); + + return; } + + if (block.rows() != 1) + throw Exception(ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY, "Scalar subquery returned more than one row"); + + Block tmp_block; + while (tmp_block.rows() == 0 && executor.pull(tmp_block)) + { + } + + if (tmp_block.rows() != 0) + throw Exception(ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY, "Scalar subquery returned more than one row"); + + block = materializeBlock(block); + size_t columns = block.columns(); + + // Block scalar; + Field scalar_value; + DataTypePtr scalar_type; + + if (columns == 1) + { + auto & column = block.getByPosition(0); + /// Here we wrap type to nullable if we can. + /// It is needed cause if subquery return no rows, it's result will be Null. + /// In case of many columns, do not check it cause tuple can't be nullable. + if (!column.type->isNullable() && column.type->canBeInsideNullable()) + { + column.type = makeNullable(column.type); + column.column = makeNullable(column.column); + } + + column.column->get(0, scalar_value); + scalar_type = column.type; + } + else + { + auto tuple_column = ColumnTuple::create(block.getColumns()); + tuple_column->get(0, scalar_value); + scalar_type = std::make_shared(block.getDataTypes(), block.getNames()); + } + + auto constant_value = std::make_shared(std::move(scalar_value), std::move(scalar_type)); + if (query_node) + query_node->performConstantFolding(std::move(constant_value)); + else if (union_node) + union_node->performConstantFolding(std::move(constant_value)); } void QueryAnalyzer::mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope) @@ -1632,6 +1589,104 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_ } } +void QueryAnalyzer::validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope) +{ + const auto limit_offset_constant_value = expression_node->getConstantValueOrNull(); + if (!limit_offset_constant_value || !isNativeNumber(removeNullable(limit_offset_constant_value->getType()))) + throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, + "{} expression must be constant with numeric type. Actual {}. In scope {}", + expression_description, + expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + Field converted = convertFieldToType(limit_offset_constant_value->getValue(), DataTypeUInt64()); + if (converted.isNull()) + throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, + "{} numeric constant expression is not representable as UInt64", + expression_description); +} + +void QueryAnalyzer::validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) +{ + auto * table_node = table_expression_node->as(); + auto * table_function_node = table_expression_node->as(); + auto * query_node = table_expression_node->as(); + auto * union_node = table_expression_node->as(); + + if (!table_node && !table_function_node && !query_node && !union_node) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected table expression. Expected table, table function, query or union node. Actual {}", + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + if (query_node || union_node) + { + auto table_expression_modifiers = query_node ? query_node->getTableExpressionModifiers() : union_node->getTableExpressionModifiers(); + + if (table_expression_modifiers.has_value()) + { + String table_expression_modifiers_error_message; + + if (table_expression_modifiers->hasFinal()) + { + table_expression_modifiers_error_message += "FINAL"; + + if (table_expression_modifiers->hasSampleSizeRatio()) + table_expression_modifiers_error_message += ", SAMPLE"; + } + else if (table_expression_modifiers->hasSampleSizeRatio()) + { + table_expression_modifiers_error_message += "SAMPLE"; + } + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Table expression modifiers {} are not supported for subquery {}. In scope {}", + table_expression_modifiers_error_message, + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + } + else if (table_node || table_function_node) + { + auto table_expression_modifiers = table_node ? table_node->getTableExpressionModifiers() : table_function_node->getTableExpressionModifiers(); + + if (table_expression_modifiers.has_value()) + { + const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); + if (table_expression_modifiers->hasFinal() && !storage->supportsFinal()) + throw Exception(ErrorCodes::ILLEGAL_FINAL, + "Storage {} doesn't support FINAL", + storage->getName()); + + if (table_expression_modifiers->hasSampleSizeRatio() && !storage->supportsSampling()) + throw Exception(ErrorCodes::SAMPLING_NOT_SUPPORTED, + "Storage {} doesn't support sampling", + storage->getStorageID().getFullNameNotQuoted()); + } + } +} + +void QueryAnalyzer::validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) +{ + if (!context->getSettingsRef().joined_subquery_requires_alias) + return; + + bool table_expression_has_alias = table_expression_node->hasAlias(); + if (table_expression_has_alias) + return; + + auto table_expression_node_type = table_expression_node->getNodeType(); + if (table_expression_node_type == QueryTreeNodeType::TABLE_FUNCTION || + table_expression_node_type == QueryTreeNodeType::QUERY || + table_expression_node_type == QueryTreeNodeType::UNION) + throw Exception(ErrorCodes::ALIAS_REQUIRED, + "JOIN {} no alias for subquery or table function {}. In scope {} (set joined_subquery_requires_alias = 0 to disable restriction)", + join_node->formatASTForErrorMessage(), + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); +} + + /// Resolve identifier functions implementation /// Try resolve table identifier from database catalog @@ -1640,7 +1695,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con size_t parts_size = table_identifier.getPartsSize(); if (parts_size < 1 || parts_size > 2) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Table identifier should consist of 1 or 2 parts. Actual {}", + "Expected table identifier to contain 1 or 2 parts. Actual '{}'", table_identifier.getFullName()); std::string database_name; @@ -1667,7 +1722,10 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con /** Resolve identifier from expression arguments. * * Expression arguments can be initialized during lambda analysis or they could be provided externally. - * Expression arguments must be already resolved nodes. This is client responsibility to resolve them during scope initialization. + * Expression arguments must be already resolved nodes. This is client responsibility to resolve them. + * + * Example: SELECT arrayMap(x -> x + 1, [1,2,3]); + * For lambda x -> x + 1, `x` is lambda expression argument. * * Resolve strategy: * 1. Try to bind identifier to scope argument name to node map. @@ -1694,13 +1752,11 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(cons } auto node_type = it->second->getNodeType(); - if (identifier_lookup.isExpressionLookup() && node_type != QueryTreeNodeType::COLUMN && node_type != QueryTreeNodeType::CONSTANT - && node_type != QueryTreeNodeType::FUNCTION && node_type != QueryTreeNodeType::QUERY && node_type != QueryTreeNodeType::UNION) + if (identifier_lookup.isExpressionLookup() && !isExpressionNodeType(node_type)) return {}; - else if (identifier_lookup.isTableExpressionLookup() && node_type != QueryTreeNodeType::TABLE && node_type != QueryTreeNodeType::TABLE_FUNCTION && - node_type != QueryTreeNodeType::QUERY && node_type != QueryTreeNodeType::UNION) + else if (identifier_lookup.isTableExpressionLookup() && !isTableExpressionNodeType(node_type)) return {}; - else if (identifier_lookup.isFunctionLookup() && node_type != QueryTreeNodeType::LAMBDA) + else if (identifier_lookup.isFunctionLookup() && !isFunctionExpressionNodeType(node_type)) return {}; if (!resolve_full_identifier && identifier_lookup.identifier.isCompound() && identifier_lookup.isExpressionLookup()) @@ -1820,7 +1876,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier if (!isNodePartOfTree(top_expression, root_expression)) throw Exception(ErrorCodes::CYCLIC_ALIASES, - "Cyclic aliases for identifier {}. In scope {}", + "Cyclic aliases for identifier '{}'. In scope {}", identifier_lookup.identifier.getFullName(), scope.scope_node->formatASTForErrorMessage()); @@ -1861,20 +1917,38 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier } else if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) { - resolveExpressionNode(it->second, scope, false /*allow_lambda_expression*/, identifier_lookup.isTableExpressionLookup() /*allow_table_expression*/); + if (identifier_resolve_settings.allow_to_resolve_subquery_during_identifier_resolution) + resolveExpressionNode(it->second, scope, false /*allow_lambda_expression*/, identifier_lookup.isTableExpressionLookup() /*allow_table_expression*/); } QueryTreeNodePtr result = it->second; - if (identifier_lookup.identifier.isCompound() && identifier_lookup.isExpressionLookup() && result) + /** If identifier is compound and it is expression identifier lookup, wrap compound expression into + * tuple elements functions. + * + * Example: SELECT compound_expression AS alias, alias.first.second; + * Result: SELECT compound_expression AS alias, tupleElement(tupleElement(compound_expression, 'first'), 'second'); + */ + if (identifier_lookup.identifier.isCompound() && result) { - auto nested_path = IdentifierView(identifier_lookup.identifier); - nested_path.popFirst(); + if (identifier_lookup.isExpressionLookup()) + { + auto nested_path = IdentifierView(identifier_lookup.identifier); + nested_path.popFirst(); - auto tuple_element_result = wrapExpressionNodeInTupleElement(result, nested_path); - resolveFunction(tuple_element_result, scope); + auto tuple_element_result = wrapExpressionNodeInTupleElement(result, nested_path); + resolveFunction(tuple_element_result, scope); - result = tuple_element_result; + result = tuple_element_result; + } + else if (identifier_lookup.isFunctionLookup() || identifier_lookup.isTableExpressionLookup()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Compound identifier '{}' cannot be resolved as {}. In scope {}", + identifier_lookup.identifier.getFullName(), + identifier_lookup.isFunctionLookup() ? "function" : "table expression", + scope.scope_node->formatASTForErrorMessage()); + } } return result; @@ -1953,7 +2027,7 @@ bool QueryAnalyzer::tryBindIdentifierToTableExpression(const IdentifierLookup & size_t parts_size = identifier_lookup.identifier.getPartsSize(); if (parts_size != 1 && parts_size != 2) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Expected identifier {} to contain 1 or 2 parts size to be resolved as table. In scope {}", + "Expected identifier '{}' to contain 1 or 2 parts to be resolved as table expression. In scope {}", identifier_lookup.identifier.getFullName(), table_expression_node->formatASTForErrorMessage()); @@ -1992,9 +2066,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id table_expression_node_type != QueryTreeNodeType::QUERY && table_expression_node_type != QueryTreeNodeType::UNION) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Unexpected table expression. Expected table, table function, query or union node. Actual {}. In scope {}", - table_expression_node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); + "Unexpected table expression. Expected table, table function, query or union node. Actual {}. In scope {}", + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); const auto & identifier = identifier_lookup.identifier; const auto & path_start = identifier.getParts().front(); @@ -2006,7 +2080,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id size_t parts_size = identifier_lookup.identifier.getPartsSize(); if (parts_size != 1 && parts_size != 2) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Expected identifier {} to contain 1 or 2 parts size to be resolved as table. In scope {}", + "Expected identifier '{}' to contain 1 or 2 parts to be resolved as table expression. In scope {}", identifier_lookup.identifier.getFullName(), table_expression_node->formatASTForErrorMessage()); @@ -2055,7 +2129,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id if (!result_column || (!match_full_identifier && !compound_identifier)) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Identifier {} cannot be resolved from {}{}. In scope {}", + "Identifier '{}' cannot be resolved from {}{}. In scope {}", identifier.getFullName(), table_expression_data.table_expression_description, table_expression_data.table_expression_name.empty() ? "" : " with name " + table_expression_data.table_expression_name, @@ -2185,21 +2259,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo std::optional resolved_side; QueryTreeNodePtr resolved_identifier; - auto resolve_from_using_column = [&](const QueryTreeNodePtr & using_column, JoinTableSide expression_side) - { - auto & using_column_node = using_column->as(); - auto & using_expression_list = using_column_node.getExpression()->as(); - - size_t inner_column_node_index = expression_side == JoinTableSide::Left ? 0 : 1; - const auto & inner_column_node = using_expression_list.getNodes().at(inner_column_node_index); - - auto result_column_node = inner_column_node->clone(); - auto & result_column = result_column_node->as(); - result_column.setColumnType(using_column_node.getColumnType()); - - return result_column_node; - }; - JoinKind join_kind = from_join_node.getKind(); if (left_resolved_identifier && right_resolved_identifier) @@ -2212,7 +2271,17 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo && left_resolved_column.getColumnName() == right_resolved_column.getColumnName()) { JoinTableSide using_column_inner_column_table_side = isRight(join_kind) ? JoinTableSide::Right : JoinTableSide::Left; - resolved_identifier = resolve_from_using_column(using_column_node_it->second, using_column_inner_column_table_side); + auto & using_column_node = using_column_node_it->second->as(); + auto & using_expression_list = using_column_node.getExpression()->as(); + + size_t inner_column_node_index = using_column_inner_column_table_side == JoinTableSide::Left ? 0 : 1; + const auto & inner_column_node = using_expression_list.getNodes().at(inner_column_node_index); + + auto result_column_node = inner_column_node->clone(); + auto & result_column = result_column_node->as(); + result_column.setColumnType(using_column_node.getColumnType()); + + resolved_identifier = std::move(result_column_node); } else { @@ -2449,7 +2518,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const } throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Resolve identifier {} from parent scope only supported for constants and CTE. Actual {} node type {}. In scope {}", + "Resolve identifier '{}' from parent scope only supported for constants and CTE. Actual {} node type {}. In scope {}", identifier_lookup.identifier.getFullName(), resolved_identifier->formatASTForErrorMessage(), resolved_identifier->getNodeTypeName(), @@ -2465,7 +2534,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const * If identifier was resolved resolve identified lookup status will be updated. * * Steps: - * 1. Register identifier lookup in scope identifier_lookup_to_resolve_status table. + * 1. Register identifier lookup in scope identifier lookup to resolve status table. * If entry is already registered and is not resolved, that means that we have cyclic aliases for identifier. * Example: SELECT a AS b, b AS a; * Try resolve identifier in current scope: @@ -2487,12 +2556,12 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const * Same is not done for functions because function resolution is more complex, and in case of aggregate functions requires not only name * but also argument types, it is responsibility of resolve function method to handle resolution of function name. * - * 9. If identifier was not resolved remove it from identifier_lookup_to_resolve_status table. + * 9. If identifier was not resolved, or identifier caching was disabled remove it from identifier lookup to resolve status table. * * It is okay for identifier to be not resolved, in case we want first try to lookup identifier in one context, * then if there is no identifier in this context, try to lookup in another context. - * Example: Try to lookup identifier as function, if it is not found lookup as expression. - * Example: Try to lookup identifier as expression, if it is not found lookup as table. + * Example: Try to lookup identifier as expression, if it is not found, lookup as function. + * Example: Try to lookup identifier as expression, if it is not found, lookup as table. */ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, IdentifierResolveSettings identifier_resolve_settings) { @@ -2501,7 +2570,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook { if (!it->second.resolved_identifier) throw Exception(ErrorCodes::CYCLIC_ALIASES, - "Cyclic aliases for identifier {}. In scope {}", + "Cyclic aliases for identifier '{}'. In scope {}", identifier_lookup.identifier.getFullName(), scope.scope_node->formatASTForErrorMessage()); @@ -2591,6 +2660,9 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook it->second = resolve_result; + /** If identifier was not resolved, or during expression resolution identifier was explicitly added into non cached set, + * or identifier caching was disabled in resolve scope we remove identifier lookup result from identifier lookup to result table. + */ if (!resolve_result.resolved_identifier || scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup) || !scope.use_identifier_lookup_to_result_cache) @@ -3340,7 +3412,7 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod { auto & lambda = lambda_node->as(); auto & lambda_arguments_nodes = lambda.getArguments().getNodes(); - size_t lambda_argument_nodes_size = lambda_arguments_nodes.size(); + size_t lambda_arguments_nodes_size = lambda_arguments_nodes.size(); /** Register lambda as being resolved, to prevent recursive lambdas resolution. * Example: WITH (x -> x + lambda_2(x)) AS lambda_1, (x -> x + lambda_1(x)) AS lambda_2 SELECT 1; @@ -3353,12 +3425,12 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod scope.scope_node->formatASTForErrorMessage()); size_t arguments_size = lambda_arguments.size(); - if (lambda_argument_nodes_size != arguments_size) + if (lambda_arguments_nodes_size != arguments_size) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Lambda {} expect {} arguments. Actual {}. In scope {}", lambda.formatASTForErrorMessage(), arguments_size, - lambda_argument_nodes_size, + lambda_arguments_nodes_size, scope.scope_node->formatASTForErrorMessage()); /// Initialize aliases in lambda scope @@ -3369,10 +3441,10 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod * Additionally validate that there are no aliases with same name as lambda arguments. * Arguments are registered in current scope expression_argument_name_to_node map. */ - auto lambda_new_arguments = std::make_shared(); - lambda_new_arguments->getNodes().reserve(lambda_argument_nodes_size); + QueryTreeNodes lambda_new_arguments_nodes; + lambda_new_arguments_nodes.reserve(lambda_arguments_nodes_size); - for (size_t i = 0; i < lambda_argument_nodes_size; ++i) + for (size_t i = 0; i < lambda_arguments_nodes_size; ++i) { auto & lambda_argument_node = lambda_arguments_nodes[i]; auto & lambda_argument_node_typed = lambda_argument_node->as(); @@ -3384,17 +3456,17 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod if (has_expression_node || has_alias_node) { throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Alias name {} inside lambda cannot have same name as lambda argument. In scope {}", + "Alias name '{}' inside lambda {} cannot have same name as lambda argument. In scope {}", lambda_argument_name, lambda_argument_node_typed.formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } scope.expression_argument_name_to_node.emplace(lambda_argument_name, lambda_arguments[i]); - lambda_new_arguments->getNodes().push_back(lambda_arguments[i]); + lambda_new_arguments_nodes.push_back(lambda_arguments[i]); } - lambda.getArgumentsNode() = std::move(lambda_new_arguments); + lambda.getArguments().getNodes() = std::move(lambda_new_arguments_nodes); /// Lambda body expression is resolved as standard query expression node. auto result_projection_names = resolveExpressionNode(lambda.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); @@ -3999,7 +4071,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } /** Resolve expression node. - * Argument node can be replaced with different node, or even with list node in case of mather resolution. + * Argument node can be replaced with different node, or even with list node in case of matcher resolution. * Example: SELECT * FROM test_table; * * - is matcher node, and it can be resolved into ListNode. * @@ -4146,7 +4218,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id message_clarification = std::string(" or ") + toStringLowercase(IdentifierLookupContext::TABLE_EXPRESSION); throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Unknown {}{} identifier {} in scope {}", + "Unknown {}{} identifier '{}' in scope {}", toStringLowercase(IdentifierLookupContext::EXPRESSION), message_clarification, unresolved_identifier.getFullName(), @@ -4161,7 +4233,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id return resolved_expression_it->second; else throw Exception(ErrorCodes::LOGICAL_ERROR, - "Identifier {} resolve into list node and list node projection names are not initialized. In scope {}", + "Identifier '{}' resolve into list node and list node projection names are not initialized. In scope {}", unresolved_identifier.getFullName(), scope.scope_node->formatASTForErrorMessage()); } @@ -4176,13 +4248,6 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id result_projection_names = resolveMatcher(node, scope); break; } - case QueryTreeNodeType::TRANSFORMER: - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Transformer {} appeared in expression context. In scope {}", - node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - } case QueryTreeNodeType::LIST: { /** Edge case if list expression has alias. @@ -4228,7 +4293,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id { if (!allow_lambda_expression) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Lambda is not allowed {} in expression. In scope {}", + "Lambda {} is not allowed in expression context. In scope {}", node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); @@ -4238,47 +4303,6 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id /// Lambda must be resolved by caller break; } - case QueryTreeNodeType::SORT: - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Sort {} is not allowed in expression. In scope {}", - node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - } - case QueryTreeNodeType::INTERPOLATE: - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Interpolate {} is not allowed in expression. In scope {}", - node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - } - case QueryTreeNodeType::WINDOW: - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Window {} is not allowed in expression. In scope {}", - node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - } - case QueryTreeNodeType::TABLE: - { - if (!allow_table_expression) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Table is not allowed {} in expression. In scope {}", - node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - - auto & table_node = node->as(); - result_projection_names.push_back(table_node.getStorageID().getFullNameNotQuoted()); - - break; - } - case QueryTreeNodeType::TABLE_FUNCTION: - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Table function is not allowed {} in expression. In scope {}", - node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - } case QueryTreeNodeType::QUERY: [[fallthrough]]; case QueryTreeNodeType::UNION: @@ -4300,17 +4324,36 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id break; } - case QueryTreeNodeType::ARRAY_JOIN: + case QueryTreeNodeType::TABLE: { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Array join {} is not allowed in expression. In scope {}", - node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); + if (!allow_table_expression) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Table {} is not allowed in expression context. In scope {}", + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + auto & table_node = node->as(); + result_projection_names.push_back(table_node.getStorageID().getFullNameNotQuoted()); + + break; } + case QueryTreeNodeType::TRANSFORMER: + [[fallthrough]]; + case QueryTreeNodeType::SORT: + [[fallthrough]]; + case QueryTreeNodeType::INTERPOLATE: + [[fallthrough]]; + case QueryTreeNodeType::WINDOW: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + [[fallthrough]]; + case QueryTreeNodeType::ARRAY_JOIN: + [[fallthrough]]; case QueryTreeNodeType::JOIN: { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Join {} is not allowed in expression. In scope {}", + "{} {} is not allowed in expression context. In scope {}", + node->getNodeType(), node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); } @@ -4546,11 +4589,7 @@ NamesAndTypes QueryAnalyzer::resolveProjectionExpressionNodeList(QueryTreeNodePt { auto projection_node = projection_nodes[i]; - if (projection_node->getNodeType() != QueryTreeNodeType::CONSTANT && - projection_node->getNodeType() != QueryTreeNodeType::FUNCTION && - projection_node->getNodeType() != QueryTreeNodeType::COLUMN && - projection_node->getNodeType() != QueryTreeNodeType::QUERY && - projection_node->getNodeType() != QueryTreeNodeType::UNION) + if (!isExpressionNodeType(projection_node->getNodeType())) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Projection node must be constant, function, column, query or union"); @@ -4563,7 +4602,7 @@ NamesAndTypes QueryAnalyzer::resolveProjectionExpressionNodeList(QueryTreeNodePt /** Initialize query join tree node. * * 1. Resolve identifiers. - * 2. Register table, table function, query nodes in scope table expressions in resolve process. + * 2. Register table, table function, query, union, join, array join nodes in scope table expressions in resolve process. */ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope) { @@ -4588,14 +4627,19 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod IdentifierResolveSettings resolve_settings; /// In join tree initialization ignore join tree as identifier lookup source resolve_settings.allow_to_check_join_tree = false; + /** Disable resolve of subquery during identifier resolution. + * Example: SELECT * FROM (SELECT 1) AS t1, t1; + * During `t1` identifier resolution we resolve it into subquery SELECT 1, but we want to disable + * subquery resolution at this stage, because JOIN TREE of parent query is not resolved. + */ + resolve_settings.allow_to_resolve_subquery_during_identifier_resolution = false; auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope, resolve_settings); auto resolved_identifier = table_identifier_resolve_result.resolved_identifier; if (!resolved_identifier) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Unknown {} identifier {} in scope {}", - toStringLowercase(IdentifierLookupContext::TABLE_EXPRESSION), + "Unknown table expression identifier '{}' in scope {}", from_table_identifier.getIdentifier().getFullName(), scope.scope_node->formatASTForErrorMessage()); @@ -4628,7 +4672,7 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod else { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Identifier in JOIN TREE {} resolve unexpected table expression. In scope {}", + "Identifier in JOIN TREE '{}' resolved into unexpected table expression. In scope {}", from_table_identifier.getIdentifier().getFullName(), scope.scope_node->formatASTForErrorMessage()); } @@ -4707,12 +4751,6 @@ void QueryAnalyzer::initializeTableExpressionColumns(const QueryTreeNodePtr & ta TableExpressionData table_expression_data; - std::string table_expression_name; - std::string table_expression_description; - - std::string table_name; - std::string database_name; - if (table_node) { const auto & table_storage_id = table_node->getStorageID(); @@ -4744,9 +4782,16 @@ void QueryAnalyzer::initializeTableExpressionColumns(const QueryTreeNodePtr & ta const auto & columns_description = storage_snapshot->metadata->getColumns(); std::vector> alias_columns_to_resolve; - std::unordered_map column_name_to_column_node; + ColumnNameToColumnNodeMap column_name_to_column_node; column_name_to_column_node.reserve(column_names_and_types.size()); + /** For ALIAS columns in table we must additionally analyze ALIAS expressions. + * Example: CREATE TABLE test_table (id UInt64, alias_value_1 ALIAS id + 5); + * + * To do that we collect alias columns and build table column name to column node map. + * For each alias column we build identifier resolve scope, initialize it with table column name to node map + * and resolve alias column. + */ for (const auto & column_name_and_type : column_names_and_types) { const auto & column_default = columns_description.getDefault(column_name_and_type.name); @@ -4778,7 +4823,6 @@ void QueryAnalyzer::initializeTableExpressionColumns(const QueryTreeNodePtr & ta /// Initialize aliases in alias column scope QueryExpressionsAliasVisitor visitor(alias_column_resolve_scope); - visitor.visit(alias_column_to_resolve->getExpression()); resolveExpressionNode(alias_column_resolve_scope.scope_node, @@ -4989,14 +5033,14 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierLookup identifier_lookup {identifier_node->getIdentifier(), IdentifierLookupContext::EXPRESSION}; auto result_left_table_expression = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_node.getLeftTableExpression(), scope); if (!result_left_table_expression) - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "JOIN {} using identifier {} cannot be resolved from left table expression. In scope {}", + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "JOIN {} using identifier '{}' cannot be resolved from left table expression. In scope {}", join_node.formatASTForErrorMessage(), identifier_full_name, scope.scope_node->formatASTForErrorMessage()); auto result_right_table_expression = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_node.getRightTableExpression(), scope); if (!result_right_table_expression) - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "JOIN {} using identifier {} cannot be resolved from right table expression. In scope {}", + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "JOIN {} using identifier '{}' cannot be resolved from right table expression. In scope {}", join_node.formatASTForErrorMessage(), identifier_full_name, scope.scope_node->formatASTForErrorMessage()); @@ -5005,7 +5049,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, if (!common_type) throw Exception(ErrorCodes::NO_COMMON_TYPE, - "JOIN {} cannot infer common type in USING for identifier {}. In scope {}", + "JOIN {} cannot infer common type in USING for identifier '{}'. In scope {}", join_node.formatASTForErrorMessage(), identifier_full_name, scope.scope_node->formatASTForErrorMessage()); @@ -5162,37 +5206,6 @@ private: const IdentifierResolveScope & scope; }; -class ValidateGroupingFunctionNodesVisitor : public ConstInDepthQueryTreeVisitor -{ -public: - explicit ValidateGroupingFunctionNodesVisitor(String assert_no_grouping_function_place_message_) - : assert_no_grouping_function_place_message(std::move(assert_no_grouping_function_place_message_)) - {} - - void visitImpl(const QueryTreeNodePtr & node) - { - auto * function_node = node->as(); - if (function_node && function_node->getFunctionName() == "grouping") - throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, - "GROUPING function {} is found {} in query", - function_node->formatASTForErrorMessage(), - assert_no_grouping_function_place_message); - } - - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) - { - return child_node->getNodeType() != QueryTreeNodeType::QUERY || child_node->getNodeType() != QueryTreeNodeType::UNION; - } - -private: - String assert_no_grouping_function_place_message; -}; - -void assertNoGroupingFunction(const QueryTreeNodePtr & node, const String & assert_no_grouping_function_place_message) -{ - ValidateGroupingFunctionNodesVisitor visitor(assert_no_grouping_function_place_message); - visitor.visit(node); -} /** Resolve query. * This function modifies query node during resolve. It is caller responsibility to clone query node before resolve @@ -5202,16 +5215,17 @@ void assertNoGroupingFunction(const QueryTreeNodePtr & node, const String & asse * scope - query scope. It is caller responsibility to create it. * * Resolve steps: - * 1. Initialize query scope with aliases. - * 2. Register CTE subqueries from WITH section in scope and remove them from WITH section. - * 3. Resolve FROM section. - * 4. Resolve projection columns. - * 5. Resolve expressions in other query parts. - * 6. Validate nodes with duplicate aliases. - * 7. Validate aggregates, aggregate functions, GROUPING function, window functions. - * 8. Remove WITH and WINDOW sections from query. - * 9. Remove aliases from expression and lambda nodes. - * 10. Resolve query tree node with projection columns. + * 1. Validate subqueries depth, perform GROUP BY validation that does not depend on information about aggregate functions. + * 2. Initialize query scope with aliases. + * 3. Register CTE subqueries from WITH section in scope and remove them from WITH section. + * 4. Resolve JOIN TREE. + * 5. Resolve projection columns. + * 6. Resolve expressions in other query parts. + * 7. Validate nodes with duplicate aliases. + * 8. Validate aggregate functions, GROUPING function, window functions. + * 9. Remove WITH and WINDOW sections from query. + * 10. Remove aliases from expression and lambda nodes. + * 11. Resolve query tree node with projection columns. */ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope) { @@ -5223,6 +5237,26 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier auto & query_node_typed = query_node->as(); + if (context->getSettingsRef().group_by_use_nulls) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "GROUP BY use nulls is not supported"); + + bool is_rollup_or_cube = query_node_typed.isGroupByWithRollup() || query_node_typed.isGroupByWithCube(); + + if (query_node_typed.isGroupByWithGroupingSets() && query_node_typed.isGroupByWithTotals()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS and GROUPING SETS are not supported together"); + + if (query_node_typed.isGroupByWithGroupingSets() && is_rollup_or_cube) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "GROUPING SETS are not supported together with ROLLUP and CUBE"); + + if (query_node_typed.isGroupByWithRollup() && (query_node_typed.isGroupByWithGroupingSets() || query_node_typed.isGroupByWithCube())) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ROLLUP is not supported together with GROUPING SETS and CUBE"); + + if (query_node_typed.isGroupByWithCube() && (query_node_typed.isGroupByWithGroupingSets() || query_node_typed.isGroupByWithRollup())) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "CUBE is not supported together with GROUPING SETS and ROLLUP"); + + if (query_node_typed.hasHaving() && query_node_typed.isGroupByWithTotals() && is_rollup_or_cube) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING"); + /// Initialize aliases in query node scope QueryExpressionsAliasVisitor visitor(scope); @@ -5582,28 +5616,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier validate_group_by_columns_visitor.visit(query_node_typed.getProjectionNode()); } - if (context->getSettingsRef().group_by_use_nulls) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "GROUP BY use nulls is not supported"); - - bool is_rollup_or_cube = query_node_typed.isGroupByWithRollup() || query_node_typed.isGroupByWithCube(); if (!has_aggregation && (query_node_typed.isGroupByWithGroupingSets() || is_rollup_or_cube)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS, ROLLUP, CUBE or GROUPING SETS are not supported without aggregation"); - if (query_node_typed.isGroupByWithGroupingSets() && query_node_typed.isGroupByWithTotals()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS and GROUPING SETS are not supported together"); - - if (query_node_typed.isGroupByWithGroupingSets() && is_rollup_or_cube) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "GROUPING SETS are not supported together with ROLLUP and CUBE"); - - if (query_node_typed.isGroupByWithRollup() && (query_node_typed.isGroupByWithGroupingSets() || query_node_typed.isGroupByWithCube())) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ROLLUP is not supported together with GROUPING SETS and CUBE"); - - if (query_node_typed.isGroupByWithCube() && (query_node_typed.isGroupByWithGroupingSets() || query_node_typed.isGroupByWithRollup())) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "CUBE is not supported together with GROUPING SETS and ROLLUP"); - - if (query_node_typed.hasHaving() && query_node_typed.isGroupByWithTotals() && is_rollup_or_cube) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING"); - /** WITH section can be safely removed, because WITH section only can provide aliases to query expressions * and CTE for other sections to use. * diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index dc3eb66ea7a..52dcb966864 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -27,7 +27,6 @@ namespace ErrorCodes * TODO: Support _shard_num into shardNum() rewriting. * TODO: Support logical expressions optimizer. * TODO: Support fuse sum count optimize_fuse_sum_count_avg, optimize_syntax_fuse_functions. - * TODO: Support setting aggregate_functions_null_for_empty. * TODO: Support setting optimize_functions_to_subcolumns. * TODO: Support setting optimize_arithmetic_operations_in_aggregate_functions. * TODO: Support setting convert_query_to_cnf. @@ -47,7 +46,6 @@ namespace ErrorCodes * TODO: Remove duplicated elements from USING clause. * TODO: Support settings.optimize_syntax_fuse_functions. * TODO: Support settings.optimize_or_like_chain. - * TODO: Support function name normalizer. * TODO: Add optimizations based on function semantics. Example: SELECT * FROM test_table WHERE id != id. (id is not nullable column). */ diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 9640694933b..ca898a7c39a 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -83,7 +83,7 @@ static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expre table_expression_node_ast = std::make_shared(identifier[0], identifier[1]); else throw Exception(ErrorCodes::LOGICAL_ERROR, - "Identifier for table expression must contain 1 or 2 parts. Actual {}", + "Identifier for table expression must contain 1 or 2 parts. Actual '{}'", identifier.getFullName()); } else diff --git a/src/Analyzer/CollectWindowFunctionNodes.cpp b/src/Analyzer/WindowFunctionsUtils.cpp similarity index 97% rename from src/Analyzer/CollectWindowFunctionNodes.cpp rename to src/Analyzer/WindowFunctionsUtils.cpp index 89c2e8ee626..e61dc4bf41d 100644 --- a/src/Analyzer/CollectWindowFunctionNodes.cpp +++ b/src/Analyzer/WindowFunctionsUtils.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Analyzer/CollectWindowFunctionNodes.h b/src/Analyzer/WindowFunctionsUtils.h similarity index 100% rename from src/Analyzer/CollectWindowFunctionNodes.h rename to src/Analyzer/WindowFunctionsUtils.h diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 88e80df5c6e..c41f79303d5 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -45,8 +45,8 @@ #include #include #include -#include -#include +#include +#include #include #include diff --git a/src/Planner/PlannerAggregation.cpp b/src/Planner/PlannerAggregation.cpp index 569b8525b3b..aabf186b49c 100644 --- a/src/Planner/PlannerAggregation.cpp +++ b/src/Planner/PlannerAggregation.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 3b85bc68e85..36bd777cf5a 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -7,8 +7,8 @@ #include #include #include -#include -#include +#include +#include #include #include diff --git a/tests/queries/0_stateless/02385_analyzer_aliases_compound_expression.reference b/tests/queries/0_stateless/02385_analyzer_aliases_compound_expression.reference new file mode 100644 index 00000000000..05c5c9872a6 --- /dev/null +++ b/tests/queries/0_stateless/02385_analyzer_aliases_compound_expression.reference @@ -0,0 +1,7 @@ +(1,'Value') 1 Value +-- +2 +-- +1 1 +-- +1 1 diff --git a/tests/queries/0_stateless/02385_analyzer_aliases_compound_expression.sql b/tests/queries/0_stateless/02385_analyzer_aliases_compound_expression.sql new file mode 100644 index 00000000000..ec2155d241b --- /dev/null +++ b/tests/queries/0_stateless/02385_analyzer_aliases_compound_expression.sql @@ -0,0 +1,21 @@ +SET use_analyzer = 1; + +SELECT cast(tuple(1, 'Value'), 'Tuple(first UInt64, second String)') AS value, value.first, value.second; + +SELECT '--'; + +WITH (x -> x + 1) AS lambda SELECT lambda(1); + +WITH (x -> x + 1) AS lambda SELECT lambda.nested(1); -- { serverError 36 } + +SELECT '--'; + +SELECT * FROM (SELECT 1) AS t1, t1 AS t2; + +SELECT '--'; + +SELECT * FROM t1 AS t2, (SELECT 1) AS t1; + +SELECT * FROM (SELECT 1) AS t1, t1.nested AS t2; -- { serverError 36 } + +SELECT * FROM t1.nested AS t2, (SELECT 1) AS t1; -- { serverError 36 } From 3a22e3769191e1a9192a06a71b5697b6cffb5353 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 11 Oct 2022 15:50:42 +0200 Subject: [PATCH 130/188] Fix normalization of UNION inside DESCRIBE query --- src/Analyzer/QueryAnalysisPass.cpp | 4 ++-- src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp | 2 +- src/Parsers/ParserDescribeTableQuery.cpp | 3 ++- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index cc62782c0e2..1e298cadf39 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -2286,9 +2286,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo else { throw Exception(ErrorCodes::AMBIGUOUS_IDENTIFIER, - "JOIN {} ambiguous identifier {}. In scope {}", + "JOIN {} ambiguous identifier '{}'. In scope {}", table_expression_node->formatASTForErrorMessage(), - identifier_lookup.dump(), + identifier_lookup.identifier.getFullName(), scope.scope_node->formatASTForErrorMessage()); } } diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp index a0fdafc976c..b3c2063c6f6 100644 --- a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp @@ -45,7 +45,7 @@ void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, SelectUnionModesSet current_set_of_modes; bool distinct_found = false; - for (ssize_t i = union_modes.size() - 1; i >= 0; --i) + for (Int64 i = union_modes.size() - 1; i >= 0; --i) { current_set_of_modes.insert(union_modes[i]); if (const auto * union_ast = typeid_cast(select_list[i + 1].get())) diff --git a/src/Parsers/ParserDescribeTableQuery.cpp b/src/Parsers/ParserDescribeTableQuery.cpp index 0f768e22324..ad6d2c5bcc6 100644 --- a/src/Parsers/ParserDescribeTableQuery.cpp +++ b/src/Parsers/ParserDescribeTableQuery.cpp @@ -33,7 +33,8 @@ bool ParserDescribeTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ex if (!ParserTableExpression().parse(pos, table_expression, expected)) return false; - query->table_expression = table_expression; + query->children.push_back(std::move(table_expression)); + query->table_expression = query->children.back(); node = query; From eba96076963af0b3f168a6abf4d27386fa4ceef5 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 11 Oct 2022 16:14:33 +0200 Subject: [PATCH 131/188] Analyzer move passes into separate folder --- src/Analyzer/{ => Passes}/CountDistinctPass.cpp | 2 +- src/Analyzer/{ => Passes}/CountDistinctPass.h | 2 +- .../{ => Passes}/CustomizeFunctionsPass.cpp | 2 +- .../{ => Passes}/CustomizeFunctionsPass.h | 0 .../{ => Passes}/IfChainToMultiIfPass.cpp | 2 +- src/Analyzer/{ => Passes}/IfChainToMultiIfPass.h | 2 +- .../{ => Passes}/IfConstantConditionPass.cpp | 2 +- .../{ => Passes}/IfConstantConditionPass.h | 2 +- src/Analyzer/{ => Passes}/MultiIfToIfPass.cpp | 2 +- src/Analyzer/{ => Passes}/MultiIfToIfPass.h | 2 +- .../{ => Passes}/NormalizeCountVariantsPass.cpp | 2 +- .../{ => Passes}/NormalizeCountVariantsPass.h | 4 ++-- .../{ => Passes}/OrderByTupleEliminationPass.cpp | 2 +- .../{ => Passes}/OrderByTupleEliminationPass.h | 0 src/Analyzer/{ => Passes}/QueryAnalysisPass.cpp | 2 +- src/Analyzer/{ => Passes}/QueryAnalysisPass.h | 0 src/Analyzer/QueryTreePassManager.cpp | 16 ++++++++-------- src/CMakeLists.txt | 1 + 18 files changed, 23 insertions(+), 22 deletions(-) rename src/Analyzer/{ => Passes}/CountDistinctPass.cpp (98%) rename src/Analyzer/{ => Passes}/CountDistinctPass.h (89%) rename src/Analyzer/{ => Passes}/CustomizeFunctionsPass.cpp (99%) rename src/Analyzer/{ => Passes}/CustomizeFunctionsPass.h (100%) rename src/Analyzer/{ => Passes}/IfChainToMultiIfPass.cpp (98%) rename src/Analyzer/{ => Passes}/IfChainToMultiIfPass.h (93%) rename src/Analyzer/{ => Passes}/IfConstantConditionPass.cpp (96%) rename src/Analyzer/{ => Passes}/IfConstantConditionPass.h (83%) rename src/Analyzer/{ => Passes}/MultiIfToIfPass.cpp (95%) rename src/Analyzer/{ => Passes}/MultiIfToIfPass.h (89%) rename src/Analyzer/{ => Passes}/NormalizeCountVariantsPass.cpp (97%) rename src/Analyzer/{ => Passes}/NormalizeCountVariantsPass.h (80%) rename src/Analyzer/{ => Passes}/OrderByTupleEliminationPass.cpp (96%) rename src/Analyzer/{ => Passes}/OrderByTupleEliminationPass.h (100%) rename src/Analyzer/{ => Passes}/QueryAnalysisPass.cpp (99%) rename src/Analyzer/{ => Passes}/QueryAnalysisPass.h (100%) diff --git a/src/Analyzer/CountDistinctPass.cpp b/src/Analyzer/Passes/CountDistinctPass.cpp similarity index 98% rename from src/Analyzer/CountDistinctPass.cpp rename to src/Analyzer/Passes/CountDistinctPass.cpp index 35ceff30337..60d44e13e7f 100644 --- a/src/Analyzer/CountDistinctPass.cpp +++ b/src/Analyzer/Passes/CountDistinctPass.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Analyzer/CountDistinctPass.h b/src/Analyzer/Passes/CountDistinctPass.h similarity index 89% rename from src/Analyzer/CountDistinctPass.h rename to src/Analyzer/Passes/CountDistinctPass.h index 6bb0aee5e1a..0de5b3a9f13 100644 --- a/src/Analyzer/CountDistinctPass.h +++ b/src/Analyzer/Passes/CountDistinctPass.h @@ -5,7 +5,7 @@ namespace DB { -/** Optimize single countDistinct into count over subquery. +/** Optimize single `countDistinct` into `count` over subquery. * * Example: SELECT countDistinct(column) FROM table; * Result: SELECT count() FROM (SELECT column FROM table GROUP BY column); diff --git a/src/Analyzer/CustomizeFunctionsPass.cpp b/src/Analyzer/Passes/CustomizeFunctionsPass.cpp similarity index 99% rename from src/Analyzer/CustomizeFunctionsPass.cpp rename to src/Analyzer/Passes/CustomizeFunctionsPass.cpp index 298c8afb516..6990b7d9d20 100644 --- a/src/Analyzer/CustomizeFunctionsPass.cpp +++ b/src/Analyzer/Passes/CustomizeFunctionsPass.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Analyzer/CustomizeFunctionsPass.h b/src/Analyzer/Passes/CustomizeFunctionsPass.h similarity index 100% rename from src/Analyzer/CustomizeFunctionsPass.h rename to src/Analyzer/Passes/CustomizeFunctionsPass.h diff --git a/src/Analyzer/IfChainToMultiIfPass.cpp b/src/Analyzer/Passes/IfChainToMultiIfPass.cpp similarity index 98% rename from src/Analyzer/IfChainToMultiIfPass.cpp rename to src/Analyzer/Passes/IfChainToMultiIfPass.cpp index 4413c8f0cc8..3fb773b070b 100644 --- a/src/Analyzer/IfChainToMultiIfPass.cpp +++ b/src/Analyzer/Passes/IfChainToMultiIfPass.cpp @@ -1,4 +1,4 @@ -#include +#include #include diff --git a/src/Analyzer/IfChainToMultiIfPass.h b/src/Analyzer/Passes/IfChainToMultiIfPass.h similarity index 93% rename from src/Analyzer/IfChainToMultiIfPass.h rename to src/Analyzer/Passes/IfChainToMultiIfPass.h index 6764592514a..dc8fa36b54b 100644 --- a/src/Analyzer/IfChainToMultiIfPass.h +++ b/src/Analyzer/Passes/IfChainToMultiIfPass.h @@ -5,7 +5,7 @@ namespace DB { -/** Convert if chain into multiIf. +/** Convert `if` chain into single `multiIf`. * Replace if(cond_1, then_1_value, if(cond_2, ...)) chains into multiIf(cond_1, then_1_value, cond_2, ...). * * Example: SELECT if(cond_1, then_1_value, if(cond_2, then_2_value, else_value)); diff --git a/src/Analyzer/IfConstantConditionPass.cpp b/src/Analyzer/Passes/IfConstantConditionPass.cpp similarity index 96% rename from src/Analyzer/IfConstantConditionPass.cpp rename to src/Analyzer/Passes/IfConstantConditionPass.cpp index 85c46b6b821..d114819ce86 100644 --- a/src/Analyzer/IfConstantConditionPass.cpp +++ b/src/Analyzer/Passes/IfConstantConditionPass.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Analyzer/IfConstantConditionPass.h b/src/Analyzer/Passes/IfConstantConditionPass.h similarity index 83% rename from src/Analyzer/IfConstantConditionPass.h rename to src/Analyzer/Passes/IfConstantConditionPass.h index 88d62c9bb48..89fcddefcfa 100644 --- a/src/Analyzer/IfConstantConditionPass.h +++ b/src/Analyzer/Passes/IfConstantConditionPass.h @@ -5,7 +5,7 @@ namespace DB { -/** Convert if with constant condition or multiIf into true condition argument value +/** Convert `if` with constant condition or `multiIf` with single constant condition into true condition argument value * or false condition argument value. * Example: SELECT if(1, true_value, false_value); * Result: SELECT true_value; diff --git a/src/Analyzer/MultiIfToIfPass.cpp b/src/Analyzer/Passes/MultiIfToIfPass.cpp similarity index 95% rename from src/Analyzer/MultiIfToIfPass.cpp rename to src/Analyzer/Passes/MultiIfToIfPass.cpp index 0bd460ae5c7..777ac3e90d2 100644 --- a/src/Analyzer/MultiIfToIfPass.cpp +++ b/src/Analyzer/Passes/MultiIfToIfPass.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Analyzer/MultiIfToIfPass.h b/src/Analyzer/Passes/MultiIfToIfPass.h similarity index 89% rename from src/Analyzer/MultiIfToIfPass.h rename to src/Analyzer/Passes/MultiIfToIfPass.h index 66fd9fd027d..d8200a03ea6 100644 --- a/src/Analyzer/MultiIfToIfPass.h +++ b/src/Analyzer/Passes/MultiIfToIfPass.h @@ -5,7 +5,7 @@ namespace DB { -/** Convert multiIf with single argument into if. +/** Convert `multiIf` with single argument into `if`. * Example: SELECT multiIf(x, 1, 0); * Result: SELECT if(x, 1, 0); */ diff --git a/src/Analyzer/NormalizeCountVariantsPass.cpp b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp similarity index 97% rename from src/Analyzer/NormalizeCountVariantsPass.cpp rename to src/Analyzer/Passes/NormalizeCountVariantsPass.cpp index 0c467592532..c7e70d4b848 100644 --- a/src/Analyzer/NormalizeCountVariantsPass.cpp +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Analyzer/NormalizeCountVariantsPass.h b/src/Analyzer/Passes/NormalizeCountVariantsPass.h similarity index 80% rename from src/Analyzer/NormalizeCountVariantsPass.h rename to src/Analyzer/Passes/NormalizeCountVariantsPass.h index 761997c5bb1..ac9f8ab9537 100644 --- a/src/Analyzer/NormalizeCountVariantsPass.h +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.h @@ -5,8 +5,8 @@ namespace DB { -/** Convert if with constant condition or multiIf into true condition argument value - * or false condition argument value. +/** Remove single literal argument from `count`. Convert `sum` with single `1` literal argument into `count`. + * * Example: SELECT count(1) * Result: SELECT count(); * diff --git a/src/Analyzer/OrderByTupleEliminationPass.cpp b/src/Analyzer/Passes/OrderByTupleEliminationPass.cpp similarity index 96% rename from src/Analyzer/OrderByTupleEliminationPass.cpp rename to src/Analyzer/Passes/OrderByTupleEliminationPass.cpp index d25e7ee39e1..14ea7c31416 100644 --- a/src/Analyzer/OrderByTupleEliminationPass.cpp +++ b/src/Analyzer/Passes/OrderByTupleEliminationPass.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Analyzer/OrderByTupleEliminationPass.h b/src/Analyzer/Passes/OrderByTupleEliminationPass.h similarity index 100% rename from src/Analyzer/OrderByTupleEliminationPass.h rename to src/Analyzer/Passes/OrderByTupleEliminationPass.h diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp similarity index 99% rename from src/Analyzer/QueryAnalysisPass.cpp rename to src/Analyzer/Passes/QueryAnalysisPass.cpp index 1e298cadf39..c79d87d11d8 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Analyzer/QueryAnalysisPass.h b/src/Analyzer/Passes/QueryAnalysisPass.h similarity index 100% rename from src/Analyzer/QueryAnalysisPass.h rename to src/Analyzer/Passes/QueryAnalysisPass.h diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 52dcb966864..3f735fb57e3 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -1,13 +1,13 @@ #include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 2c54d1108d4..ce2cc862b32 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -257,6 +257,7 @@ add_object_library(clickhouse_databases Databases) add_object_library(clickhouse_databases_mysql Databases/MySQL) add_object_library(clickhouse_disks Disks) add_object_library(clickhouse_analyzer Analyzer) +add_object_library(clickhouse_analyzer_passes Analyzer/Passes) add_object_library(clickhouse_planner Planner) add_object_library(clickhouse_interpreters Interpreters) add_object_library(clickhouse_interpreters_cache Interpreters/Cache) From 7851dfc3242d4934ba7f079030fd03e9dff6ad88 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 11 Oct 2022 18:54:23 +0200 Subject: [PATCH 132/188] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 28 +++++----- .../02366_explain_query_tree.reference | 14 ++--- .../02378_analyzer_projection_names.reference | 53 +++++++++++++++++-- .../02378_analyzer_projection_names.sql | 37 +++++++++++-- ...2382_analyzer_matcher_join_using.reference | 1 + .../02382_analyzer_matcher_join_using.sql | 2 + 6 files changed, 102 insertions(+), 33 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index c79d87d11d8..34f585dc322 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -181,7 +181,7 @@ namespace ErrorCodes * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. * TODO: Support function identifier resolve from parent query scope, if lambda in parent scope does not capture any columns. - * TODO: Support group_by_use_nulls + * TODO: Support group_by_use_nulls. * TODO: Scalar subqueries cache. */ @@ -5022,6 +5022,8 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, if (join_node.isUsingJoinExpression()) { auto & join_using_list = join_node.getJoinExpression()->as(); + std::unordered_set join_using_identifiers; + for (auto & join_using_node : join_using_list.getNodes()) { auto * identifier_node = join_using_node->as(); @@ -5030,6 +5032,14 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, const auto & identifier_full_name = identifier_node->getIdentifier().getFullName(); + if (join_using_identifiers.contains(identifier_full_name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "JOIN {} identifier '{}' appears more than once in USING clause", + join_node.formatASTForErrorMessage(), + identifier_full_name); + + join_using_identifiers.insert(identifier_full_name); + IdentifierLookup identifier_lookup {identifier_node->getIdentifier(), IdentifierLookupContext::EXPRESSION}; auto result_left_table_expression = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_node.getLeftTableExpression(), scope); if (!result_left_table_expression) @@ -5060,16 +5070,6 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, join_using_node = std::move(join_using_column); } - - for (auto & join_using_node : join_using_list.getNodes()) - { - if (!join_using_node->as()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "JOIN {} USING identifier node must be resolved into column node. Actual {}. In scope {}", - join_node.formatASTForErrorMessage(), - join_tree_node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - } } else if (join_node.getJoinExpression()) { @@ -5098,10 +5098,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, } auto join_tree_node_type = join_tree_node->getNodeType(); - if (join_tree_node_type == QueryTreeNodeType::QUERY || - join_tree_node_type == QueryTreeNodeType::UNION || - join_tree_node_type == QueryTreeNodeType::TABLE || - join_tree_node_type == QueryTreeNodeType::TABLE_FUNCTION) + if (isTableExpressionNodeType(join_tree_node_type)) { validateTableExpressionModifiers(join_tree_node, scope); initializeTableExpressionColumns(join_tree_node, scope); @@ -5206,7 +5203,6 @@ private: const IdentifierResolveScope & scope; }; - /** Resolve query. * This function modifies query node during resolve. It is caller responsibility to clone query node before resolve * if it is needed for later use. diff --git a/tests/queries/0_stateless/02366_explain_query_tree.reference b/tests/queries/0_stateless/02366_explain_query_tree.reference index 36cb367d946..769d7661e68 100644 --- a/tests/queries/0_stateless/02366_explain_query_tree.reference +++ b/tests/queries/0_stateless/02366_explain_query_tree.reference @@ -1,11 +1,11 @@ -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary +QUERY id: 0 PROJECTION LIST id: 1, nodes: 1 CONSTANT id: 2, constant_value: UInt64_1, constant_value_type: UInt8 JOIN TREE IDENTIFIER id: 3, identifier: system.one -- -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary +QUERY id: 0 PROJECTION LIST id: 1, nodes: 2 IDENTIFIER id: 2, identifier: id @@ -13,7 +13,7 @@ QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, i JOIN TREE IDENTIFIER id: 4, identifier: test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary +QUERY id: 0 PROJECTION COLUMNS id UInt64 value String @@ -24,7 +24,7 @@ QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, i JOIN TREE TABLE id: 3, table_name: default.test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary +QUERY id: 0 PROJECTION LIST id: 1, nodes: 1 FUNCTION id: 2, function_name: arrayMap, function_type: ordinary @@ -44,7 +44,7 @@ QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, i JOIN TREE IDENTIFIER id: 12, identifier: test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary +QUERY id: 0 PROJECTION COLUMNS arrayMap(lambda(tuple(x), plus(x, 1)), [1, 2, 3]) Array(UInt16) PROJECTION @@ -66,7 +66,7 @@ QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, i JOIN TREE TABLE id: 11, table_name: default.test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary +QUERY id: 0 WITH LIST id: 1, nodes: 1 LAMBDA id: 2, alias: lambda @@ -88,7 +88,7 @@ QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, i JOIN TREE IDENTIFIER id: 13, identifier: test_table -- -QUERY id: 0, is_subquery: 0, is_cte: 0, is_distinct: 0, is_limit_with_ties: 0, is_group_by_with_totals: 0, group_by_type: ordinary +QUERY id: 0 PROJECTION COLUMNS lambda(id) UInt64 PROJECTION diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.reference b/tests/queries/0_stateless/02378_analyzer_projection_names.reference index ea16b2f127b..1fa79677876 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.reference +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.reference @@ -288,6 +288,52 @@ DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b) AS c, c.a, c.b); c Tuple(a UInt8, b UInt8) c.a UInt8 c.b UInt8 +SELECT '--'; +-- +DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b) AS c, c.*); +c Tuple(a UInt8, b UInt8) +c.a UInt8 +c.b UInt8 +SELECT '--'; +-- +DESCRIBE (SELECT (SELECT 1 UNION DISTINCT SELECT 1), (SELECT 2 UNION DISTINCT SELECT 2), (SELECT 3 UNION DISTINCT SELECT 3) AS a, (SELECT 4 UNION DISTINCT SELECT 4)); +_subquery_1 Nullable(UInt8) +_subquery_2 Nullable(UInt8) +a Nullable(UInt8) +_subquery_4 Nullable(UInt8) +SELECT '--'; +-- +DESCRIBE (SELECT arrayMap(x -> (SELECT 1 UNION DISTINCT SELECT 1), [1,2,3]), arrayMap(x -> (SELECT 2 UNION DISTINCT SELECT 2) AS a, [1, 2, 3]), +arrayMap(x -> (SELECT 3 UNION DISTINCT SELECT 3), [1,2,3])); +arrayMap(lambda(tuple(x), _subquery_1), [1, 2, 3]) Array(Nullable(UInt8)) +arrayMap(a, [1, 2, 3]) Array(Nullable(UInt8)) +arrayMap(lambda(tuple(x), _subquery_3), [1, 2, 3]) Array(Nullable(UInt8)) +SELECT '--'; +-- +DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b UNION DISTINCT SELECT 1, 2) AS c, c.a, c.b); +c Tuple(a UInt8, b UInt8) +c.a UInt8 +c.b UInt8 +SELECT '--'; +-- +DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b UNION DISTINCT SELECT 1, 2) AS c, c.*); +c Tuple(a UInt8, b UInt8) +c.a UInt8 +c.b UInt8 +SELECT '--'; +-- +DESCRIBE (SELECT (SELECT 1), (SELECT 2 UNION DISTINCT SELECT 2), (SELECT 3) AS a, (SELECT 4 UNION DISTINCT SELECT 4)); +_subquery_1 Nullable(UInt8) +_subquery_2 Nullable(UInt8) +a Nullable(UInt8) +_subquery_4 Nullable(UInt8) +SELECT '--'; +-- +DESCRIBE (SELECT arrayMap(x -> (SELECT 1 UNION DISTINCT SELECT 1), [1,2,3]), arrayMap(x -> (SELECT 2) AS a, [1, 2, 3]), +arrayMap(x -> (SELECT 3 UNION DISTINCT SELECT 3), [1,2,3])); +arrayMap(lambda(tuple(x), _subquery_1), [1, 2, 3]) Array(Nullable(UInt8)) +arrayMap(a, [1, 2, 3]) Array(Nullable(UInt8)) +arrayMap(lambda(tuple(x), _subquery_3), [1, 2, 3]) Array(Nullable(UInt8)) SELECT 'Window functions'; Window functions DESCRIBE (SELECT count() OVER ()); @@ -342,7 +388,8 @@ DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DES count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN plus(1, 1) PRECEDING AND plus(2, 2) FOLLOWING) UInt64 SELECT '--'; -- -DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN ((1 + 1) AS frame_offset_begin) PRECEDING AND ((2 + 2) AS frame_offset_end) FOLLOWING) FROM test_table); +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN ((1 + 1) AS frame_offset_begin) PRECEDING AND ((2 + 2) AS frame_offset_end) FOLLOWING) +FROM test_table); count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN frame_offset_begin PRECEDING AND frame_offset_end FOLLOWING) UInt64 SELECT '--'; -- @@ -354,10 +401,6 @@ DESCRIBE (SELECT count() OVER (ORDER BY toNullable(id) NULLS LAST) FROM test_tab count() OVER (ORDER BY toNullable(id) ASC NULLS LAST) UInt64 SELECT '--'; -- -DESCRIBE (SELECT count() OVER (ORDER BY value COLLATE 'EN') FROM test_table); -count() OVER (ORDER BY value ASC COLLATE en) UInt64 -SELECT '--'; --- DESCRIBE (SELECT count() OVER (ORDER BY id WITH FILL FROM 1 TO 5 STEP 1) FROM test_table); count() OVER (ORDER BY id ASC WITH FILL FROM 1 TO 5 STEP 1) UInt64 SELECT '--'; diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.sql b/tests/queries/0_stateless/02378_analyzer_projection_names.sql index 5703f63deea..a31f5afd8fb 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.sql +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.sql @@ -268,6 +268,36 @@ SELECT '--'; DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b) AS c, c.a, c.b); +SELECT '--'; + +DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b) AS c, c.*); + +SELECT '--'; + +DESCRIBE (SELECT (SELECT 1 UNION DISTINCT SELECT 1), (SELECT 2 UNION DISTINCT SELECT 2), (SELECT 3 UNION DISTINCT SELECT 3) AS a, (SELECT 4 UNION DISTINCT SELECT 4)); + +SELECT '--'; + +DESCRIBE (SELECT arrayMap(x -> (SELECT 1 UNION DISTINCT SELECT 1), [1,2,3]), arrayMap(x -> (SELECT 2 UNION DISTINCT SELECT 2) AS a, [1, 2, 3]), +arrayMap(x -> (SELECT 3 UNION DISTINCT SELECT 3), [1,2,3])); + +SELECT '--'; + +DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b UNION DISTINCT SELECT 1, 2) AS c, c.a, c.b); + +SELECT '--'; + +DESCRIBE (SELECT (SELECT 1 AS a, 2 AS b UNION DISTINCT SELECT 1, 2) AS c, c.*); + +SELECT '--'; + +DESCRIBE (SELECT (SELECT 1), (SELECT 2 UNION DISTINCT SELECT 2), (SELECT 3) AS a, (SELECT 4 UNION DISTINCT SELECT 4)); + +SELECT '--'; + +DESCRIBE (SELECT arrayMap(x -> (SELECT 1 UNION DISTINCT SELECT 1), [1,2,3]), arrayMap(x -> (SELECT 2) AS a, [1, 2, 3]), +arrayMap(x -> (SELECT 3 UNION DISTINCT SELECT 3), [1,2,3])); + SELECT 'Window functions'; DESCRIBE (SELECT count() OVER ()); @@ -322,7 +352,8 @@ DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DES SELECT '--'; -DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN ((1 + 1) AS frame_offset_begin) PRECEDING AND ((2 + 2) AS frame_offset_end) FOLLOWING) FROM test_table); +DESCRIBE (SELECT count() OVER (PARTITION BY id, value ORDER BY id ASC, value DESC ROWS BETWEEN ((1 + 1) AS frame_offset_begin) PRECEDING AND ((2 + 2) AS frame_offset_end) FOLLOWING) +FROM test_table); SELECT '--'; @@ -334,10 +365,6 @@ DESCRIBE (SELECT count() OVER (ORDER BY toNullable(id) NULLS LAST) FROM test_tab SELECT '--'; -DESCRIBE (SELECT count() OVER (ORDER BY value COLLATE 'EN') FROM test_table); - -SELECT '--'; - DESCRIBE (SELECT count() OVER (ORDER BY id WITH FILL FROM 1 TO 5 STEP 1) FROM test_table); SELECT '--'; diff --git a/tests/queries/0_stateless/02382_analyzer_matcher_join_using.reference b/tests/queries/0_stateless/02382_analyzer_matcher_join_using.reference index 3f34634cbe8..f2199aad4c8 100644 --- a/tests/queries/0_stateless/02382_analyzer_matcher_join_using.reference +++ b/tests/queries/0_stateless/02382_analyzer_matcher_join_using.reference @@ -3,6 +3,7 @@ SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) ORDER BY id, t1.value; 0 Join_1_Value_0 Join_2_Value_0 1 Join_1_Value_1 Join_2_Value_1 +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id, id, id) ORDER BY id, t1.value; -- { serverError 36 } SELECT '--'; -- SELECT * FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) ORDER BY id, t1.value; diff --git a/tests/queries/0_stateless/02382_analyzer_matcher_join_using.sql b/tests/queries/0_stateless/02382_analyzer_matcher_join_using.sql index 29a4708aeea..8b6d9832086 100644 --- a/tests/queries/0_stateless/02382_analyzer_matcher_join_using.sql +++ b/tests/queries/0_stateless/02382_analyzer_matcher_join_using.sql @@ -37,6 +37,8 @@ INSERT INTO test_table_join_3 VALUES (4, 'Join_3_Value_4'); SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) ORDER BY id, t1.value; +SELECT * FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id, id, id) ORDER BY id, t1.value; -- { serverError 36 } + SELECT '--'; SELECT * FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) ORDER BY id, t1.value; From da85af7e44ad3aa48d4919c5f212fc45612fe2ee Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 11 Oct 2022 18:54:38 +0200 Subject: [PATCH 133/188] Added OrderByLimitByDuplicateEliminationPass --- ...OrderByLimitByDuplicateEliminationPass.cpp | 99 +++++++++++++++++++ .../OrderByLimitByDuplicateEliminationPass.h | 27 +++++ src/Analyzer/QueryTreePassManager.cpp | 5 +- 3 files changed, 128 insertions(+), 3 deletions(-) create mode 100644 src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.cpp create mode 100644 src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h diff --git a/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.cpp b/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.cpp new file mode 100644 index 00000000000..6b84f376115 --- /dev/null +++ b/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.cpp @@ -0,0 +1,99 @@ +#include + +#include +#include +#include + +namespace DB +{ + +namespace +{ + +struct QueryTreeNodeHash +{ + size_t operator()(const IQueryTreeNode * node) const + { + return node->getTreeHash().first; + } +}; + +struct QueryTreeNodeEqualTo +{ + size_t operator()(const IQueryTreeNode * lhs_node, const IQueryTreeNode * rhs_node) const + { + return lhs_node->isEqual(*rhs_node); + } +}; + +using QueryTreeNodeSet = std::unordered_set; + +class OrderByLimitByDuplicateEliminationVisitor : public InDepthQueryTreeVisitor +{ +public: + void visitImpl(QueryTreeNodePtr & node) + { + auto * query_node = node->as(); + if (!query_node) + return; + + if (query_node->hasOrderBy()) + { + QueryTreeNodes result_nodes; + + auto & query_order_by_nodes = query_node->getOrderBy().getNodes(); + + for (auto & sort_node : query_order_by_nodes) + { + auto & sort_node_typed = sort_node->as(); + + /// Skip elements with WITH FILL + if (sort_node_typed.withFill()) + { + result_nodes.push_back(sort_node); + continue; + } + + auto [_, inserted] = unique_expressions_nodes_set.emplace(sort_node_typed.getExpression().get()); + if (inserted) + result_nodes.push_back(sort_node); + } + + query_order_by_nodes = std::move(result_nodes); + } + + unique_expressions_nodes_set.clear(); + + if (query_node->hasLimitBy()) + { + QueryTreeNodes result_nodes; + + auto & query_limit_by_nodes = query_node->getLimitBy().getNodes(); + + for (auto & limit_by_node : query_node->getLimitBy().getNodes()) + { + auto [_, inserted] = unique_expressions_nodes_set.emplace(limit_by_node.get()); + if (inserted) + result_nodes.push_back(limit_by_node); + } + + query_limit_by_nodes = std::move(result_nodes); + } + + unique_expressions_nodes_set.clear(); + } + +private: + QueryTreeNodeSet unique_expressions_nodes_set; +}; + +} + +void OrderByLimitByDuplicateEliminationPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) +{ + OrderByLimitByDuplicateEliminationVisitor visitor; + visitor.visit(query_tree_node); +} + +} + diff --git a/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h b/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h new file mode 100644 index 00000000000..b50099dbc6a --- /dev/null +++ b/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h @@ -0,0 +1,27 @@ +#pragma once + +#include + +namespace DB +{ + +/** Eliminate duplicate columns from ORDER BY and LIMIT BY. + * Example: SELECT * FROM test_table ORDER BY id, id; + * Result: SELECT * FROM test_table ORDER BY id; + * + * Example: SELECT * FROM test_table LIMIT 5 BY id, id; + * Result: SELECT * FROM test_table LIMIT 5 BY id; + */ +class OrderByLimitByDuplicateEliminationPass final : public IQueryTreePass +{ +public: + String getName() override { return "OrderByLimitByDuplicateEliminationPass"; } + + String getDescription() override { return "Remove duplicate columns from ORDER BY, LIMIT BY."; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} + diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 3f735fb57e3..b5146b65f6f 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -41,9 +42,6 @@ namespace ErrorCodes * TODO: Support setting optimize_redundant_functions_in_order_by. * TODO: Support setting optimize_monotonous_functions_in_order_by. * TODO: Support setting optimize_if_transform_strings_to_enum. - * TODO: Remove duplicate elements from ORDER BY clause. - * TODO: Remove duplicated elements from LIMIT BY clause. - * TODO: Remove duplicated elements from USING clause. * TODO: Support settings.optimize_syntax_fuse_functions. * TODO: Support settings.optimize_or_like_chain. * TODO: Add optimizations based on function semantics. Example: SELECT * FROM test_table WHERE id != id. (id is not nullable column). @@ -134,6 +132,7 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_shared()); manager.addPass(std::make_shared()); + manager.addPass(std::make_shared()); } } From 5a0298ea3482805d72303d8b9ab9f4476ccb1549 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 12 Oct 2022 13:26:02 +0200 Subject: [PATCH 134/188] Fixed tests --- src/Analyzer/QueryTreeBuilder.cpp | 4 +- src/Analyzer/TableExpressionModifiers.h | 1 - src/Analyzer/Utils.cpp | 4 +- .../InterpreterSelectQueryAnalyzer.cpp | 27 ++-- .../InterpreterSelectQueryAnalyzer.h | 10 +- src/Planner/Planner.cpp | 139 +++++++++++++----- src/Planner/Planner.h | 2 + src/Planner/PlannerJoinTree.cpp | 42 ++++-- src/Planner/Utils.cpp | 50 +++++++ src/Planner/Utils.h | 3 + 10 files changed, 219 insertions(+), 63 deletions(-) diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 534999419ae..53a13ceeec1 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -445,7 +445,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co function->formatForErrorMessage()); if (lambda_argument_identifier->name_parts.size() > 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Lambda {} argument identifier must contain only argument name. Actual {}", function->formatForErrorMessage(), lambda_argument_identifier->full_name); @@ -453,7 +453,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co const auto & argument_name = lambda_argument_identifier->name_parts[0]; auto [_, inserted] = lambda_arguments_set.insert(argument_name); if (!inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Lambda {} multiple arguments with same name {}", function->formatForErrorMessage(), argument_name); diff --git a/src/Analyzer/TableExpressionModifiers.h b/src/Analyzer/TableExpressionModifiers.h index c7a65d4c2b5..ea66d62dec4 100644 --- a/src/Analyzer/TableExpressionModifiers.h +++ b/src/Analyzer/TableExpressionModifiers.h @@ -59,7 +59,6 @@ private: inline bool operator==(const TableExpressionModifiers & lhs, const TableExpressionModifiers & rhs) { - lhs.getSampleOffsetRatio(); return lhs.hasFinal() == rhs.hasFinal() && lhs.getSampleSizeRatio() == rhs.getSampleSizeRatio() && lhs.getSampleOffsetRatio() == rhs.getSampleOffsetRatio(); } diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index ca898a7c39a..adce27902a7 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -278,12 +278,12 @@ void buildTableExpressionsStackImpl(const QueryTreeNodePtr & join_tree_node, Que default: { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected node type for table expression. Expected table, table function, query, union, join or array join. Actual " - "{}", + "Unexpected node type for table expression. Expected table, table function, query, union, join or array join. Actual {}", join_tree_node->getNodeTypeName()); } } } + } QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_node) diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index bd6acb90da2..61ec5932b7d 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -14,6 +14,7 @@ #include #include +#include namespace DB { @@ -87,12 +88,6 @@ Block InterpreterSelectQueryAnalyzer::getSampleBlock() return planner.getQueryPlan().getCurrentDataStream().header; } -QueryPlan && InterpreterSelectQueryAnalyzer::extractQueryPlan() && -{ - planner.buildQueryPlanIfNeeded(); - return std::move(planner).extractQueryPlan(); -} - BlockIO InterpreterSelectQueryAnalyzer::execute() { planner.buildQueryPlanIfNeeded(); @@ -102,10 +97,24 @@ BlockIO InterpreterSelectQueryAnalyzer::execute() BuildQueryPipelineSettings build_pipeline_settings; auto pipeline_builder = query_plan.buildQueryPipeline(optimization_settings, build_pipeline_settings); - BlockIO res; - res.pipeline = QueryPipelineBuilder::getPipeline(std::move(*pipeline_builder)); + BlockIO result; + result.pipeline = QueryPipelineBuilder::getPipeline(std::move(*pipeline_builder)); - return res; + if (!select_query_options.ignore_quota && (select_query_options.to_stage == QueryProcessingStage::Complete)) + result.pipeline.setQuota(getContext()->getQuota()); + + return result; +} + +QueryPlan && InterpreterSelectQueryAnalyzer::extractQueryPlan() && +{ + planner.buildQueryPlanIfNeeded(); + return std::move(planner).extractQueryPlan(); +} + +void InterpreterSelectQueryAnalyzer::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const +{ + elem.query_kind = "Select"; } } diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 3e3e2c18e54..e9884567ab0 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -27,12 +27,18 @@ public: Block getSampleBlock(); - QueryPlan && extractQueryPlan() &&; - BlockIO execute() override; + QueryPlan && extractQueryPlan() &&; + bool supportsTransactions() const override { return true; } + bool ignoreLimits() const override { return select_query_options.ignore_limits; } + + bool ignoreQuota() const override { return select_query_options.ignore_quota; } + + void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const override; + private: ASTPtr query; QueryTreeNodePtr query_tree; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index c41f79303d5..b730399928c 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1,5 +1,7 @@ #include +#include + #include #include @@ -69,6 +71,7 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; + extern const int TOO_DEEP_SUBQUERIES; } /** ClickHouse query planner. @@ -88,11 +91,43 @@ namespace ErrorCodes * TODO: Support ORDER BY read in order optimization * TODO: Support GROUP BY read in order optimization * TODO: Support Key Condition. Support indexes for IN function. + * TODO: Better support for quota and limits. */ namespace { +/** Check that table and table function table expressions from planner context support transactions. + * + * There is precondition that table expression data for table expression nodes is collected in planner context. + */ +void checkStoragesSupportTransactions(const PlannerContextPtr & planner_context) +{ + const auto & query_context = planner_context->getQueryContext(); + if (query_context->getSettingsRef().throw_on_unsupported_query_inside_transaction) + return; + + if (!query_context->getCurrentTransaction()) + return; + + for (const auto & [table_expression, _] : planner_context->getTableExpressionNodeToData()) + { + StoragePtr storage; + if (auto * table_node = table_expression->as()) + storage = table_node->getStorage(); + else if (auto * table_function_node = table_expression->as()) + storage = table_function_node->getStorage(); + + if (storage->supportsTransactions()) + continue; + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Storage {} (table {}) does not support transactions", + storage->getName(), + storage->getStorageID().getNameForLogs()); + } +} + void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, const SelectQueryOptions & select_query_options, const PlannerContextPtr & planner_context) { if (select_query_options.is_subquery) @@ -122,6 +157,25 @@ void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, const SelectQ addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); } +/// Extend lifetime of query context, storages, and table locks +void extendQueryContextAndStoragesLifetime(QueryPlan & query_plan, const PlannerContextPtr & planner_context) +{ + query_plan.addInterpreterContext(planner_context->getQueryContext()); + + for (const auto & [table_expression, _] : planner_context->getTableExpressionNodeToData()) + { + if (auto * table_node = table_expression->as()) + { + query_plan.addStorageHolder(table_node->getStorage()); + query_plan.addTableLock(table_node->getStorageLock()); + } + else if (auto * table_function_node = table_expression->as()) + { + query_plan.addStorageHolder(table_function_node->getStorage()); + } + } +} + } Planner::Planner(const QueryTreeNodePtr & query_tree_, @@ -131,11 +185,7 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_, , select_query_options(select_query_options_) , planner_context(std::make_shared(context_, std::make_shared())) { - if (query_tree->getNodeType() != QueryTreeNodeType::QUERY && - query_tree->getNodeType() != QueryTreeNodeType::UNION) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Expected QUERY or UNION node. Actual {}", - query_tree->formatASTForErrorMessage()); + initialize(); } /// Initialize interpreter with query tree after query analysis phase and global planner context @@ -147,11 +197,26 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_, , select_query_options(select_query_options_) , planner_context(std::make_shared(context_, std::move(global_planner_context_))) { + initialize(); +} + +void Planner::initialize() +{ + checkStackSize(); + if (query_tree->getNodeType() != QueryTreeNodeType::QUERY && query_tree->getNodeType() != QueryTreeNodeType::UNION) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected QUERY or UNION node. Actual {}", query_tree->formatASTForErrorMessage()); + + const auto & query_context = planner_context->getQueryContext(); + const Settings & settings = query_context->getSettingsRef(); + + if (settings.max_subquery_depth && select_query_options.subquery_depth > settings.max_subquery_depth) + throw Exception(ErrorCodes::TOO_DEEP_SUBQUERIES, + "Too deep subqueries. Maximum: {}", + settings.max_subquery_depth.toString()); } void Planner::buildQueryPlanIfNeeded() @@ -286,6 +351,8 @@ void Planner::buildQueryPlanIfNeeded() select_query_info.storage_limits = std::make_shared(storage_limits); collectTableExpressionData(query_tree, *planner_context); + checkStoragesSupportTransactions(planner_context); + collectSets(query_tree, *planner_context); query_plan = buildQueryPlanForJoinTreeNode(query_node.getJoinTree(), select_query_info, select_query_options, planner_context); @@ -496,6 +563,21 @@ void Planner::buildQueryPlanIfNeeded() expression_step_projection->setStepDescription("Projection"); query_plan.addStep(std::move(expression_step_projection)); + UInt64 limit_offset = 0; + if (query_node.hasOffset()) + { + /// Constness of offset is validated during query analysis stage + limit_offset = query_node.getOffset()->getConstantValue().getValue().safeGet(); + } + + UInt64 limit_length = 0; + + if (query_node.hasLimit()) + { + /// Constness of limit is validated during query analysis stage + limit_length = query_node.getLimit()->getConstantValue().getValue().safeGet(); + } + if (query_node.isDistinct()) { const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); @@ -503,6 +585,13 @@ void Planner::buildQueryPlanIfNeeded() bool pre_distinct = true; SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); + bool no_order_by = !query_node.hasOrderBy(); + + /** If after this stage of DISTINCT ORDER BY is not executed, + * then you can get no more than limit_length + limit_offset of different rows. + */ + if (no_order_by && limit_length <= std::numeric_limits::max() - limit_offset) + limit_hint_for_distinct = limit_length + limit_offset; auto distinct_step = std::make_unique( query_plan.getCurrentDataStream(), @@ -535,7 +624,16 @@ void Planner::buildQueryPlanIfNeeded() { sort_description = extractSortDescription(query_node.getOrderByNode(), *planner_context); - UInt64 limit = 0; + bool query_has_array_join = queryHasArrayJoin(query_tree); + + UInt64 partial_sorting_limit = 0; + + /// Partial sort can be done if there is LIMIT, but no DISTINCT, LIMIT WITH TIES, LIMIT BY, ARRAY JOIN. + if (limit_length != 0 && !query_node.isDistinct() && !query_node.hasLimitBy() && !query_node.isLimitWithTies() && !query_has_array_join && + limit_length <= std::numeric_limits::max() - limit_offset) + { + partial_sorting_limit = limit_length + limit_offset; + } const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); @@ -544,7 +642,7 @@ void Planner::buildQueryPlanIfNeeded() query_plan.getCurrentDataStream(), sort_description, settings.max_block_size, - limit, + partial_sorting_limit, SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), settings.max_bytes_before_remerge_sort, settings.remerge_sort_lowered_memory_bytes_ratio, @@ -678,22 +776,12 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(extremes_step)); } - UInt64 limit_offset = 0; - if (query_node.hasOffset()) - { - /// Constness of offset is validated during query analysis stage - limit_offset = query_node.getOffset()->getConstantValue().getValue().safeGet(); - } - if (query_node.hasLimit()) { const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); bool always_read_till_end = settings.exact_rows_before_limit; bool limit_with_ties = query_node.isLimitWithTies(); - /// Constness of limit is validated during query analysis stage - UInt64 limit_length = query_node.getLimit()->getConstantValue().getValue().safeGet(); - SortDescription limit_with_ties_sort_description; if (query_node.isLimitWithTies()) @@ -728,22 +816,7 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(projection_step)); addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context); - - /// Extend lifetime of context, table locks, storages - query_plan.addInterpreterContext(planner_context->getQueryContext()); - - for (const auto & [table_expression, _] : planner_context->getTableExpressionNodeToData()) - { - if (auto * table_node = table_expression->as()) - { - query_plan.addStorageHolder(table_node->getStorage()); - query_plan.addTableLock(table_node->getStorageLock()); - } - else if (auto * table_function_node = table_expression->as()) - { - query_plan.addStorageHolder(table_function_node->getStorage()); - } - } + extendQueryContextAndStoragesLifetime(query_plan, planner_context); } } diff --git a/src/Planner/Planner.h b/src/Planner/Planner.h index e29b347fdfe..389df4365fa 100644 --- a/src/Planner/Planner.h +++ b/src/Planner/Planner.h @@ -48,6 +48,8 @@ public: } private: + void initialize(); + QueryTreeNodePtr query_tree; QueryPlan query_plan; SelectQueryOptions select_query_options; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 4be2d0762a7..1ae0b2ef54f 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -78,28 +78,42 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, const auto & columns_names = table_expression_data.getColumnsNames(); Names column_names(columns_names.begin(), columns_names.end()); - std::optional read_additional_column; - if (column_names.empty()) { auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); - read_additional_column = column_names_and_types.front(); + auto additional_column_to_read = column_names_and_types.front(); + + const auto & column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(additional_column_to_read, table_expression); + column_names.push_back(additional_column_to_read.name); + table_expression_data.addColumn(additional_column_to_read, column_identifier); } - if (read_additional_column) + const auto & query_context = planner_context->getQueryContext(); + size_t max_block_size = query_context->getSettingsRef().max_block_size; + size_t max_streams = query_context->getSettingsRef().max_threads; + + bool need_rewrite_query_with_final = storage->needRewriteQueryWithFinal(column_names); + if (need_rewrite_query_with_final) { - const auto & column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(*read_additional_column, table_expression); - column_names.push_back(read_additional_column->name); - table_expression_data.addColumn(*read_additional_column, column_identifier); + if (table_expression_query_info.table_expression_modifiers) + { + const auto & table_expression_modifiers = table_expression_query_info.table_expression_modifiers; + auto sample_size_ratio = table_expression_modifiers->getSampleSizeRatio(); + auto sample_offset_ratio = table_expression_modifiers->getSampleOffsetRatio(); + + table_expression_query_info.table_expression_modifiers = TableExpressionModifiers(true /*has_final*/, + sample_size_ratio, + sample_offset_ratio); + } + else + { + table_expression_query_info.table_expression_modifiers = TableExpressionModifiers(true /*has_final*/, + {} /*sample_size_ratio*/, + {} /*sample_offset_ratio*/); + } } - if (!column_names.empty()) - { - const auto & query_context = planner_context->getQueryContext(); - size_t max_block_size = query_context->getSettingsRef().max_block_size; - size_t max_streams = query_context->getSettingsRef().max_threads; - storage->read(query_plan, column_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); - } + storage->read(query_plan, column_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); /// Create step which reads from empty source if storage has no data. if (!query_plan.isInitialized()) diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 12b2b244965..3f4a532b462 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include @@ -194,5 +195,54 @@ bool sortDescriptionIsPrefix(const SortDescription & prefix, const SortDescripti return true; } +bool queryHasArrayJoin(const QueryTreeNodePtr & query_node) +{ + const auto & query_node_typed = query_node->as(); + auto join_tree_node = query_node_typed.getJoinTree(); + + std::vector join_tree_nodes_to_process; + join_tree_nodes_to_process.push_back(std::move(join_tree_node)); + + while (join_tree_nodes_to_process.empty()) + { + auto join_tree_node_to_process = join_tree_nodes_to_process.back(); + join_tree_nodes_to_process.pop_back(); + + auto join_tree_node_type = join_tree_node_to_process->getNodeType(); + + switch (join_tree_node_type) + { + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::UNION: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + return true; + } + case QueryTreeNodeType::JOIN: + { + auto & join_node = join_tree_node->as(); + join_tree_nodes_to_process.push_back(join_node.getLeftTableExpression()); + join_tree_nodes_to_process.push_back(join_node.getRightTableExpression()); + break; + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected node type for table expression. Expected table, table function, query, union, join or array join. Actual {}", + join_tree_node_to_process->getNodeTypeName()); + } + } + } + + return false; +} } diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 35913e1096f..5be2f75dd17 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -48,4 +48,7 @@ ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & express /// Returns true if prefix sort description is prefix of full sort descriptor, false otherwise bool sortDescriptionIsPrefix(const SortDescription & prefix, const SortDescription & full); +/// Returns true if query node JOIN TREE contains ARRAY JOIN node +bool queryHasArrayJoin(const QueryTreeNodePtr & query_node); + } From b50e2ce47075265ab883dca97fbf710eb3fdb56e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 12 Oct 2022 15:46:50 +0200 Subject: [PATCH 135/188] Added SETTINGS support --- src/Analyzer/InterpolateNode.h | 3 +- src/Analyzer/Passes/QueryAnalysisPass.cpp | 102 +++++----- src/Analyzer/QueryNode.cpp | 10 +- src/Analyzer/QueryNode.h | 216 ++++++++++++++++++---- src/Analyzer/QueryTreeBuilder.cpp | 8 + src/Planner/Planner.cpp | 42 ++++- src/Planner/PlannerContext.h | 6 + 7 files changed, 297 insertions(+), 90 deletions(-) diff --git a/src/Analyzer/InterpolateNode.h b/src/Analyzer/InterpolateNode.h index ba7ef47756c..5bc8eded0bb 100644 --- a/src/Analyzer/InterpolateNode.h +++ b/src/Analyzer/InterpolateNode.h @@ -7,7 +7,8 @@ namespace DB { /** Interpolate node represents expression interpolation in INTERPOLATE section that is part of ORDER BY section in query tree. - * Example: SELECT * FROM test_table ORDER BY id WITH FILL INTERPOLATE value AS value + 1; + * + * Example: SELECT * FROM test_table ORDER BY id WITH FILL INTERPOLATE (value AS value + 1); * value - expression to interpolate. * value + 1 - interpolate expression. */ diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 34f585dc322..86ffc9b7d44 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -634,13 +634,18 @@ struct IdentifierResolveScope , parent_scope(parent_scope_) { if (parent_scope) + { subquery_depth = parent_scope->subquery_depth; + context = parent_scope->context; + } } QueryTreeNodePtr scope_node; IdentifierResolveScope * parent_scope = nullptr; + ContextPtr context; + /// Identifier lookup to result std::unordered_map identifier_lookup_to_result; @@ -957,13 +962,10 @@ private: class QueryAnalyzer { public: - explicit QueryAnalyzer(ContextPtr context_) - : context(std::move(context_)) - {} - - void resolve(QueryTreeNodePtr node, const QueryTreeNodePtr & table_expression) + void resolve(QueryTreeNodePtr node, const QueryTreeNodePtr & table_expression, ContextPtr context) { IdentifierResolveScope scope(node, nullptr /*parent_scope*/); + scope.context = context; auto node_type = node->getNodeType(); @@ -1051,9 +1053,9 @@ private: static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path); - QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); + static QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunctions(const std::string & function_name, ContextPtr context); - void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, size_t subquery_depth); + static void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, size_t subquery_depth, ContextPtr context); static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope); @@ -1063,11 +1065,11 @@ private: static void validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); - void validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + static void validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); /// Resolve identifier functions - QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier); + static QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier, ContextPtr context); QueryTreeNodePtr tryResolveIdentifierFromExpressionArguments(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); @@ -1135,9 +1137,6 @@ private: void resolveUnion(const QueryTreeNodePtr & union_node, IdentifierResolveScope & scope); - /// Query analyzer context - ContextPtr context; - /// Lambdas that are currently in resolve process std::unordered_set lambdas_in_resolve_process; @@ -1377,7 +1376,7 @@ QueryTreeNodePtr QueryAnalyzer::wrapExpressionNodeInTupleElement(QueryTreeNodePt /** Try to get lambda node from sql user defined functions if sql user defined function with function name exists. * Returns lambda node if function exists, nullptr otherwise. */ -QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name) +QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunctions(const std::string & function_name, ContextPtr context) { auto user_defined_function = UserDefinedSQLFunctionFactory::instance().tryGet(function_name); if (!user_defined_function) @@ -1395,7 +1394,7 @@ QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunction(const std } /// Evaluate scalar subquery and perform constant folding if scalar subquery does not have constant value -void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size_t subquery_depth) +void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size_t subquery_depth, ContextPtr context) { auto * query_node = node->as(); auto * union_node = node->as(); @@ -1409,6 +1408,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size return; auto subquery_context = Context::createCopy(context); + Settings subquery_settings = context->getSettings(); subquery_settings.max_result_rows = 1; subquery_settings.extremes = false; @@ -1668,7 +1668,7 @@ void QueryAnalyzer::validateTableExpressionModifiers(const QueryTreeNodePtr & ta void QueryAnalyzer::validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) { - if (!context->getSettingsRef().joined_subquery_requires_alias) + if (!scope.context->getSettingsRef().joined_subquery_requires_alias) return; bool table_expression_has_alias = table_expression_node->hasAlias(); @@ -1690,7 +1690,7 @@ void QueryAnalyzer::validateJoinTableExpressionWithoutAlias(const QueryTreeNodeP /// Resolve identifier functions implementation /// Try resolve table identifier from database catalog -QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier) +QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier, ContextPtr context) { size_t parts_size = table_identifier.getPartsSize(); if (parts_size < 1 || parts_size > 2) @@ -1716,6 +1716,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con auto storage = DatabaseCatalog::instance().getTable(storage_id, context); auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); + return std::make_shared(std::move(storage), storage_lock, storage_snapshot); } @@ -2160,7 +2161,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id if (qualified_identifier_with_removed_part.empty()) break; - if (context->getSettingsRef().prefer_column_name_to_alias + if (scope.context->getSettingsRef().prefer_column_name_to_alias && scope.alias_name_to_expression_node.contains(qualified_identifier_with_removed_part[0])) break; @@ -2334,7 +2335,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo if (join_node_in_resolve_process || !resolved_identifier) return resolved_identifier; - bool join_use_nulls = context->getSettingsRef().join_use_nulls; + bool join_use_nulls = scope.context->getSettingsRef().join_use_nulls; if (join_use_nulls && (isFull(join_kind) || @@ -2590,7 +2591,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook if (!resolve_result.resolved_identifier) { - bool prefer_column_name_to_alias = context->getSettingsRef().prefer_column_name_to_alias; + bool prefer_column_name_to_alias = scope.context->getSettingsRef().prefer_column_name_to_alias; if (unlikely(prefer_column_name_to_alias)) { @@ -2652,7 +2653,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook if (!resolve_result.resolved_identifier && identifier_resolve_settings.allow_to_check_database_catalog && identifier_lookup.isTableExpressionLookup()) { - resolve_result.resolved_identifier = tryResolveTableIdentifierFromDatabaseCatalog(identifier_lookup.identifier); + resolve_result.resolved_identifier = tryResolveTableIdentifierFromDatabaseCatalog(identifier_lookup.identifier, scope.context); if (resolve_result.resolved_identifier) resolve_result.resolve_place = IdentifierResolvePlace::DATABASE_CATALOG; @@ -2986,11 +2987,12 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( if (matcher_node_typed.isAsteriskMatcher()) { get_column_options_kind = GetColumnsOptions::Ordinary; + const auto & settings = scope.context->getSettingsRef(); - if (context->getSettingsRef().asterisk_include_alias_columns) + if (settings.asterisk_include_alias_columns) get_column_options_kind |= GetColumnsOptions::Kind::Aliases; - if (context->getSettingsRef().asterisk_include_materialized_columns) + if (settings.asterisk_include_materialized_columns) get_column_options_kind |= GetColumnsOptions::Kind::Materialized; } else @@ -3747,7 +3749,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi if (!function_node.isWindowFunction()) { if (!lambda_expression_untyped) - lambda_expression_untyped = tryGetLambdaFromSQLUserDefinedFunction(function_node.getFunctionName()); + lambda_expression_untyped = tryGetLambdaFromSQLUserDefinedFunctions(function_node.getFunctionName(), scope.context); /** If function is resolved as lambda. * Clone lambda before resolve. @@ -3840,7 +3842,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi "Function GROUPING can have up to 64 arguments, but {} provided", function_arguments_size); - bool force_grouping_standard_compatibility = context->getSettingsRef().force_grouping_standard_compatibility; + bool force_grouping_standard_compatibility = scope.context->getSettingsRef().force_grouping_standard_compatibility; auto grouping_function = std::make_shared(force_grouping_standard_compatibility); auto grouping_function_adaptor = std::make_shared(std::move(grouping_function)); function_node.resolveAsFunction(std::move(grouping_function_adaptor), std::make_shared()); @@ -3872,10 +3874,10 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi return result_projection_names; } - FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, context, parameters); + FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, scope.context, parameters); if (!function) - function = FunctionFactory::instance().tryGet(function_name, context); + function = FunctionFactory::instance().tryGet(function_name, scope.context); if (!function) { @@ -4014,7 +4016,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi const auto & second_argument_constant_literal = second_argument_constant_value.getValue(); const auto & second_argument_constant_type = second_argument_constant_value.getType(); - auto set = makeSetForConstantValue(first_argument_constant_type, second_argument_constant_literal, second_argument_constant_type, context->getSettingsRef()); + auto set = makeSetForConstantValue(first_argument_constant_type, second_argument_constant_literal, second_argument_constant_type, scope.context->getSettingsRef()); /// Create constant set column for constant folding @@ -4110,7 +4112,8 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id { IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); subquery_scope.subquery_depth = scope.subquery_depth + 1; - evaluateScalarSubqueryIfNeeded(node, subquery_scope.subquery_depth); + + evaluateScalarSubqueryIfNeeded(node, subquery_scope.subquery_depth, subquery_scope.context); } return resolved_expression_it->second; @@ -4316,7 +4319,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id resolveUnion(node, subquery_scope); if (!allow_table_expression) - evaluateScalarSubqueryIfNeeded(node, subquery_scope.subquery_depth); + evaluateScalarSubqueryIfNeeded(node, subquery_scope.subquery_depth, subquery_scope.context); ++subquery_counter; if (result_projection_names.empty()) @@ -4798,7 +4801,7 @@ void QueryAnalyzer::initializeTableExpressionColumns(const QueryTreeNodePtr & ta if (column_default && column_default->kind == ColumnDefaultKind::Alias) { - auto column_node = std::make_shared(column_name_and_type, buildQueryTree(column_default->expression, context), table_expression_node); + auto column_node = std::make_shared(column_name_and_type, buildQueryTree(column_default->expression, scope.context), table_expression_node); column_name_to_column_node.emplace(column_name_and_type.name, column_node); alias_columns_to_resolve.emplace_back(column_name_and_type.name, column_node); } @@ -4820,6 +4823,7 @@ void QueryAnalyzer::initializeTableExpressionColumns(const QueryTreeNodePtr & ta IdentifierResolveScope alias_column_resolve_scope(alias_column_to_resolve, nullptr /*parent_scope*/); alias_column_resolve_scope.column_name_to_column_node = std::move(column_name_to_column_node); + alias_column_resolve_scope.context = scope.context; /// Initialize aliases in alias column scope QueryExpressionsAliasVisitor visitor(alias_column_resolve_scope); @@ -4906,7 +4910,9 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, const auto & table_function_factory = TableFunctionFactory::instance(); const auto & table_function_name = table_function_node.getTableFunctionName(); - TableFunctionPtr table_function_ptr = table_function_factory.tryGet(table_function_name, context); + auto & scope_context = scope.context; + + TableFunctionPtr table_function_ptr = table_function_factory.tryGet(table_function_name, scope_context); if (!table_function_ptr) { auto hints = TableFunctionFactory::instance().getHints(table_function_name); @@ -4919,13 +4925,13 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", table_function_name); } - if (context->getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint()) + if (scope_context->getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint()) { - const auto & insertion_table = context->getInsertionTable(); + const auto & insertion_table = scope_context->getInsertionTable(); if (!insertion_table.empty()) { const auto & structure_hint - = DatabaseCatalog::instance().getTable(insertion_table, context)->getInMemoryMetadataPtr()->columns; + = DatabaseCatalog::instance().getTable(insertion_table, scope_context)->getInMemoryMetadataPtr()->columns; table_function_ptr->setStructureHint(structure_hint); } } @@ -4951,10 +4957,11 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, } auto table_function_ast = table_function_node.toAST(); - table_function_ptr->parseArguments(table_function_ast, context); + table_function_ptr->parseArguments(table_function_ast, scope_context); + + auto table_function_storage = table_function_ptr->execute(table_function_ast, scope_context, table_function_ptr->getName()); + table_function_node.resolve(std::move(table_function_ptr), std::move(table_function_storage), scope_context); - auto table_function_storage = table_function_ptr->execute(table_function_ast, context, table_function_ptr->getName()); - table_function_node.resolve(std::move(table_function_ptr), std::move(table_function_storage), context); break; } case QueryTreeNodeType::TABLE: @@ -5225,15 +5232,24 @@ private: */ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope) { - const auto & settings = context->getSettingsRef(); - if (settings.max_subquery_depth && scope.subquery_depth > settings.max_subquery_depth) + size_t max_subquery_depth = scope.context->getSettingsRef().max_subquery_depth; + if (max_subquery_depth && scope.subquery_depth > max_subquery_depth) throw Exception(ErrorCodes::TOO_DEEP_SUBQUERIES, "Too deep subqueries. Maximum: {}", - settings.max_subquery_depth.toString()); + max_subquery_depth); auto & query_node_typed = query_node->as(); - if (context->getSettingsRef().group_by_use_nulls) + if (query_node_typed.hasSettingsChanges()) + { + auto updated_scope_context = Context::createCopy(scope.context); + updated_scope_context->applySettingsChanges(query_node_typed.getSettingsChanges()); + scope.context = std::move(updated_scope_context); + } + + const auto & settings = scope.context->getSettingsRef(); + + if (settings.group_by_use_nulls) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "GROUP BY use nulls is not supported"); bool is_rollup_or_cube = query_node_typed.isGroupByWithRollup() || query_node_typed.isGroupByWithCube(); @@ -5675,8 +5691,8 @@ QueryAnalysisPass::QueryAnalysisPass(QueryTreeNodePtr table_expression_) void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { - QueryAnalyzer analyzer(std::move(context)); - analyzer.resolve(query_tree_node, table_expression); + QueryAnalyzer analyzer; + analyzer.resolve(query_tree_node, table_expression, context); } } diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 23457256979..7cde154dc6f 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -14,8 +14,7 @@ #include #include #include -#include -#include +#include #include @@ -413,6 +412,13 @@ ASTPtr QueryNode::toASTImpl() const if (hasOffset()) select_query->setExpression(ASTSelectQuery::Expression::LIMIT_OFFSET, getOffset()->toAST()); + if (hasSettingsChanges()) + { + auto settings_query = std::make_shared(); + settings_query->changes = settings_changes; + select_query->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(settings_query)); + } + auto result_select_query = std::make_shared(); result_select_query->union_mode = SelectUnionMode::UNION_DEFAULT; diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index d82cb0f903f..e93b8604a4d 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include @@ -17,6 +19,46 @@ namespace ErrorCodes } /** Query node represents query in query tree. + * + * Example: SELECT * FROM test_table WHERE id == 0; + * Example: SELECT * FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id; + * + * Query node consists of following sections. + * 1. WITH section. + * 2. PROJECTION section. + * 3. JOIN TREE section. + * Example: SELECT * FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id; + * test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id - JOIN TREE section. + * 4. PREWHERE section. + * 5. WHERE section. + * 6. GROUP BY section. + * 7. HAVING section. + * 8. WINDOW section. + * Example: SELECT * FROM test_table WINDOW window AS (PARTITION BY id); + * 9. ORDER BY section. + * 10. INTERPOLATE section. + * Example: SELECT * FROM test_table ORDER BY id WITH FILL INTERPOLATE (value AS value + 1); + * value AS value + 1 - INTERPOLATE section. + * 11. LIMIT BY limit section. + * 12. LIMIT BY offset section. + * 13. LIMIT BY section. + * Example: SELECT * FROM test_table LIMIT 1 AS a OFFSET 5 AS b BY id, value; + * 1 AS a - LIMIT BY limit section. + * 5 AS b - LIMIT BY offset section. + * id, value - LIMIT BY section. + * 14. LIMIT section. + * 15. OFFSET section. + * + * Query node contains settings changes that must be applied before query analysis or execution. + * Example: SELECT * FROM test_table SETTINGS prefer_column_name_to_alias = 1, join_use_nulls = 1; + * + * Query node can be used as CTE. + * Example: WITH cte_subquery AS (SELECT 1) SELECT * FROM cte_subquery; + * + * Query node can be used as scalar subquery. + * Example: SELECT (SELECT 1) AS scalar_subquery. + * + * During query analysis pass query node must be resolved with projection columns. */ class QueryNode; using QueryNodePtr = std::shared_ptr; @@ -26,96 +68,114 @@ class QueryNode final : public IQueryTreeNode public: explicit QueryNode(); + /// Returns true if query node is subquery, false otherwise bool isSubquery() const { return is_subquery; } + /// Set query node is subquery void setIsSubquery(bool is_subquery_value) { is_subquery = is_subquery_value; } + /// Returns true if query node is CTE, false otherwise bool isCTE() const { return is_cte; } + /// Set query node is CTE void setIsCTE(bool is_cte_value) { is_cte = is_cte_value; } + /// Get query node CTE name const std::string & getCTEName() const { return cte_name; } + /// Set query node CTE name void setCTEName(std::string cte_name_value) { cte_name = std::move(cte_name_value); } - void setIsDistinct(bool is_distinct_value) - { - is_distinct = is_distinct_value; - } - + /// Returns true if query node has DISTINCT, false otherwise bool isDistinct() const { return is_distinct; } - void setIsLimitWithTies(bool is_limit_with_ties_value) + /// Set query node DISTINCT value + void setIsDistinct(bool is_distinct_value) { - is_limit_with_ties = is_limit_with_ties_value; + is_distinct = is_distinct_value; } + /// Returns true if query node has LIMIT WITH TIES, false otherwise bool isLimitWithTies() const { return is_limit_with_ties; } - void setIsGroupByWithTotals(bool is_group_by_with_totals_value) + /// Set query node LIMIT WITH TIES value + void setIsLimitWithTies(bool is_limit_with_ties_value) { - is_group_by_with_totals = is_group_by_with_totals_value; + is_limit_with_ties = is_limit_with_ties_value; } + /// Returns true, if query node has GROUP BY WITH TOTALS, false otherwise bool isGroupByWithTotals() const { return is_group_by_with_totals; } - void setIsGroupByWithRollup(bool is_group_by_with_rollup_value) + /// Set query node GROUP BY WITH TOTALS value + void setIsGroupByWithTotals(bool is_group_by_with_totals_value) { - is_group_by_with_rollup = is_group_by_with_rollup_value; + is_group_by_with_totals = is_group_by_with_totals_value; } + /// Returns true, if query node has GROUP BY with ROLLUP modifier, false otherwise bool isGroupByWithRollup() const { return is_group_by_with_rollup; } - void setIsGroupByWithCube(bool is_group_by_with_cube_value) + /// Set query node GROUP BY with ROLLUP modifier value + void setIsGroupByWithRollup(bool is_group_by_with_rollup_value) { - is_group_by_with_cube = is_group_by_with_cube_value; + is_group_by_with_rollup = is_group_by_with_rollup_value; } + /// Returns true, if query node has GROUP BY with CUBE modifier, false otherwise bool isGroupByWithCube() const { return is_group_by_with_cube; } - void setIsGroupByWithGroupingSets(bool is_group_by_with_grouping_sets_value) + /// Set query node GROUP BY with CUBE modifier value + void setIsGroupByWithCube(bool is_group_by_with_cube_value) { - is_group_by_with_grouping_sets = is_group_by_with_grouping_sets_value; + is_group_by_with_cube = is_group_by_with_cube_value; } + /// Returns true, if query node has GROUP BY with GROUPING SETS modifier, false otherwise bool isGroupByWithGroupingSets() const { return is_group_by_with_grouping_sets; } + /// Set query node GROUP BY with GROUPING SETS modifier value + void setIsGroupByWithGroupingSets(bool is_group_by_with_grouping_sets_value) + { + is_group_by_with_grouping_sets = is_group_by_with_grouping_sets_value; + } + /// Return true if query node has table expression modifiers, false otherwise bool hasTableExpressionModifiers() const { @@ -134,286 +194,361 @@ public: table_expression_modifiers = std::move(table_expression_modifiers_value); } + /// Returns true if query node WITH section is not empty bool hasWith() const { return !getWith().getNodes().empty(); } + /// Get WITH section const ListNode & getWith() const { return children[with_child_index]->as(); } + /// Get WITH section ListNode & getWith() { return children[with_child_index]->as(); } + /// Get WITH section node const QueryTreeNodePtr & getWithNode() const { return children[with_child_index]; } + /// Get WITH section node QueryTreeNodePtr & getWithNode() { return children[with_child_index]; } + /// Get PROJECTION section const ListNode & getProjection() const { return children[projection_child_index]->as(); } + /// Get PROJECTION section ListNode & getProjection() { return children[projection_child_index]->as(); } + /// Get PROJECTION section node const QueryTreeNodePtr & getProjectionNode() const { return children[projection_child_index]; } + /// Get PROJECTION section node QueryTreeNodePtr & getProjectionNode() { return children[projection_child_index]; } + /// Get JOIN TREE section node const QueryTreeNodePtr & getJoinTree() const { return children[join_tree_child_index]; } + /// Get JOIN TREE section node QueryTreeNodePtr & getJoinTree() { return children[join_tree_child_index]; } - bool hasWindow() const - { - return !getWindow().getNodes().empty(); - } - - const ListNode & getWindow() const - { - return children[window_child_index]->as(); - } - - ListNode & getWindow() - { - return children[window_child_index]->as(); - } - - const QueryTreeNodePtr & getWindowNode() const - { - return children[window_child_index]; - } - - QueryTreeNodePtr & getWindowNode() - { - return children[window_child_index]; - } - + /// Returns true if query node PREWHERE section is not empty bool hasPrewhere() const { return children[prewhere_child_index] != nullptr; } + /// Get PREWHERE section node const QueryTreeNodePtr & getPrewhere() const { return children[prewhere_child_index]; } + /// Get PREWHERE section node QueryTreeNodePtr & getPrewhere() { return children[prewhere_child_index]; } + /// Returns true if query node WHERE section is not empty bool hasWhere() const { return children[where_child_index] != nullptr; } + /// Get WHERE section node const QueryTreeNodePtr & getWhere() const { return children[where_child_index]; } + /// Get WHERE section node QueryTreeNodePtr & getWhere() { return children[where_child_index]; } + /// Returns true if query node GROUP BY section is not empty bool hasGroupBy() const { return !getGroupBy().getNodes().empty(); } + /// Get GROUP BY section const ListNode & getGroupBy() const { return children[group_by_child_index]->as(); } + /// Get GROUP BY section ListNode & getGroupBy() { return children[group_by_child_index]->as(); } + /// Get GROUP BY section node const QueryTreeNodePtr & getGroupByNode() const { return children[group_by_child_index]; } + /// Get GROUP BY section node QueryTreeNodePtr & getGroupByNode() { return children[group_by_child_index]; } + /// Returns true if query node HAVING section is not empty bool hasHaving() const { return getHaving() != nullptr; } + /// Get HAVING section node const QueryTreeNodePtr & getHaving() const { return children[having_child_index]; } + /// Get HAVING section node QueryTreeNodePtr & getHaving() { return children[having_child_index]; } + /// Returns true if query node WINDOW section is not empty + bool hasWindow() const + { + return !getWindow().getNodes().empty(); + } + + /// Get WINDOW section + const ListNode & getWindow() const + { + return children[window_child_index]->as(); + } + + /// Get WINDOW section + ListNode & getWindow() + { + return children[window_child_index]->as(); + } + + /// Get WINDOW section node + const QueryTreeNodePtr & getWindowNode() const + { + return children[window_child_index]; + } + + /// Get WINDOW section node + QueryTreeNodePtr & getWindowNode() + { + return children[window_child_index]; + } + + /// Returns true if query node ORDER BY section is not empty bool hasOrderBy() const { return !getOrderBy().getNodes().empty(); } + /// Get ORDER BY section const ListNode & getOrderBy() const { return children[order_by_child_index]->as(); } + /// Get ORDER BY section ListNode & getOrderBy() { return children[order_by_child_index]->as(); } + /// Get ORDER BY section node const QueryTreeNodePtr & getOrderByNode() const { return children[order_by_child_index]; } + /// Get ORDER BY section node QueryTreeNodePtr & getOrderByNode() { return children[order_by_child_index]; } + /// Returns true if query node INTERPOLATE section is not empty bool hasInterpolate() const { return getInterpolate() != nullptr; } + /// Get INTERPOLATE section node const QueryTreeNodePtr & getInterpolate() const { return children[interpolate_child_index]; } + /// Get INTERPOLATE section node QueryTreeNodePtr & getInterpolate() { return children[interpolate_child_index]; } + /// Returns true if query node LIMIT BY LIMIT section is not empty bool hasLimitByLimit() const { return children[limit_by_limit_child_index] != nullptr; } + /// Get LIMIT BY LIMIT section node const QueryTreeNodePtr & getLimitByLimit() const { return children[limit_by_limit_child_index]; } + /// Get LIMIT BY LIMIT section node QueryTreeNodePtr & getLimitByLimit() { return children[limit_by_limit_child_index]; } + /// Returns true if query node LIMIT BY OFFSET section is not empty bool hasLimitByOffset() const { return children[limit_by_offset_child_index] != nullptr; } + /// Get LIMIT BY OFFSET section node const QueryTreeNodePtr & getLimitByOffset() const { return children[limit_by_offset_child_index]; } + /// Get LIMIT BY OFFSET section node QueryTreeNodePtr & getLimitByOffset() { return children[limit_by_offset_child_index]; } + /// Returns true if query node LIMIT BY section is not empty bool hasLimitBy() const { return !getLimitBy().getNodes().empty(); } + /// Get LIMIT BY section const ListNode & getLimitBy() const { return children[limit_by_child_index]->as(); } + /// Get LIMIT BY section ListNode & getLimitBy() { return children[limit_by_child_index]->as(); } + /// Get LIMIT BY section node const QueryTreeNodePtr & getLimitByNode() const { return children[limit_by_child_index]; } + /// Get LIMIT BY section node QueryTreeNodePtr & getLimitByNode() { return children[limit_by_child_index]; } + /// Returns true if query node LIMIT section is not empty bool hasLimit() const { return children[limit_child_index] != nullptr; } + /// Get LIMIT section node const QueryTreeNodePtr & getLimit() const { return children[limit_child_index]; } + /// Get LIMIT section node QueryTreeNodePtr & getLimit() { return children[limit_child_index]; } + /// Returns true if query node OFFSET section is not empty bool hasOffset() const { return children[offset_child_index] != nullptr; } + /// Get OFFSET section node const QueryTreeNodePtr & getOffset() const { return children[offset_child_index]; } + /// Get OFFSET section node QueryTreeNodePtr & getOffset() { return children[offset_child_index]; } + /// Returns true if query node has settings changes specified, false otherwise + bool hasSettingsChanges() const + { + return !settings_changes.empty(); + } + + /// Get query node settings changes + const SettingsChanges & getSettingsChanges() const + { + return settings_changes; + } + + /// Set query node settings changes value + void setSettingsChanges(SettingsChanges settings_changes_value) + { + settings_changes = std::move(settings_changes_value); + } + + /// Get query node projection columns const NamesAndTypes & getProjectionColumns() const { return projection_columns; } + /// Resolve query node projection columns void resolveProjectionColumns(NamesAndTypes projection_columns_value) { projection_columns = std::move(projection_columns_value); @@ -470,6 +605,7 @@ private: NamesAndTypes projection_columns; ConstantValuePtr constant_value; std::optional table_expression_modifiers; + SettingsChanges settings_changes; static constexpr size_t with_child_index = 0; static constexpr size_t projection_child_index = 1; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 53a13ceeec1..09baa1e98b8 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -217,6 +218,13 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q current_query_tree->setIsGroupByWithGroupingSets(select_query_typed.group_by_with_grouping_sets); current_query_tree->setOriginalAST(select_query); + auto select_settings = select_query_typed.settings(); + if (select_settings) + { + auto & set_query = select_settings->as(); + current_query_tree->setSettingsChanges(set_query.changes); + } + current_query_tree->getJoinTree() = buildJoinTree(select_query_typed.tables()); auto select_with_list = select_query_typed.with(); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index b730399928c..40761a34c5e 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -2,6 +2,8 @@ #include +#include + #include #include @@ -72,6 +74,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int TOO_DEEP_SUBQUERIES; + extern const int NOT_IMPLEMENTED; } /** ClickHouse query planner. @@ -210,13 +213,44 @@ void Planner::initialize() "Expected QUERY or UNION node. Actual {}", query_tree->formatASTForErrorMessage()); - const auto & query_context = planner_context->getQueryContext(); - const Settings & settings = query_context->getSettingsRef(); + auto & query_context = planner_context->getQueryContext(); - if (settings.max_subquery_depth && select_query_options.subquery_depth > settings.max_subquery_depth) + size_t max_subquery_depth = query_context->getSettingsRef().max_subquery_depth; + if (max_subquery_depth && select_query_options.subquery_depth > max_subquery_depth) throw Exception(ErrorCodes::TOO_DEEP_SUBQUERIES, "Too deep subqueries. Maximum: {}", - settings.max_subquery_depth.toString()); + max_subquery_depth); + + auto * query_node = query_tree->as(); + if (!query_node) + return; + + bool need_apply_query_settings = query_node->hasSettingsChanges(); + + const auto & client_info = query_context->getClientInfo(); + auto min_major = static_cast(DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD); + auto min_minor = static_cast(DBMS_MIN_MINOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD); + + bool need_to_disable_two_level_aggregation = client_info.query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && + client_info.connection_client_version_major < min_major && + client_info.connection_client_version_minor < min_minor; + + if (need_apply_query_settings || need_to_disable_two_level_aggregation) + { + auto updated_context = Context::createCopy(query_context); + + if (need_apply_query_settings) + updated_context->applySettingsChanges(query_node->getSettingsChanges()); + + /// Disable two-level aggregation due to version incompatibility. + if (need_to_disable_two_level_aggregation) + { + updated_context->setSetting("group_by_two_level_threshold", Field(0)); + updated_context->setSetting("group_by_two_level_threshold_bytes", Field(0)); + } + + query_context = std::move(updated_context); + } } void Planner::buildQueryPlanIfNeeded() diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 3c2dfd8e863..22c746c82c9 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -96,6 +96,12 @@ public: return query_context; } + /// Get planner context query context + ContextPtr & getQueryContext() + { + return query_context; + } + /// Get global planner context const GlobalPlannerContextPtr & getGlobalPlannerContext() const { From f4e59b217f93d87c5d69e95bc0bc3e2c82072353 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 12 Oct 2022 16:16:01 +0200 Subject: [PATCH 136/188] Added RBAC support --- src/Planner/PlannerJoinTree.cpp | 43 +++++++++++++++++++++++++++++++-- 1 file changed, 41 insertions(+), 2 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 1ae0b2ef54f..9b05f78388f 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -5,6 +5,9 @@ #include #include +#include +#include + #include #include @@ -42,11 +45,40 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int SYNTAX_ERROR; + extern const int ACCESS_DENIED; } namespace { +/// Check if current user has privileges to SELECT columns from table +void checkAccessRights(const TableNode & table_node, const Names & column_names, const ContextPtr & query_context) +{ + const auto & storage_id = table_node.getStorageID(); + const auto & storage_snapshot = table_node.getStorageSnapshot(); + + if (column_names.empty()) + { + /** For a trivial queries like "SELECT count() FROM table", "SELECT 1 FROM table" access is granted if at least + * one table column is accessible. + */ + auto access = query_context->getAccess(); + + for (const auto & column : storage_snapshot->metadata->getColumns()) + { + if (access->isGranted(AccessType::SELECT, storage_id.database_name, storage_id.table_name, column.name)) + return; + } + + throw Exception(ErrorCodes::ACCESS_DENIED, + "{}: Not enough privileges. To execute this query it's necessary to have grant SELECT for at least one column on {}", + query_context->getUserName(), + storage_id.getFullTableName()); + } + + query_context->checkAccess(AccessType::SELECT, storage_id, column_names); +} + QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, @@ -74,10 +106,18 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, else table_expression_query_info.table_expression_modifiers = table_function_node->getTableExpressionModifiers(); - auto from_stage = storage->getQueryProcessingStage(planner_context->getQueryContext(), select_query_options.to_stage, storage_snapshot, table_expression_query_info); + auto & query_context = planner_context->getQueryContext(); + + auto from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); const auto & columns_names = table_expression_data.getColumnsNames(); Names column_names(columns_names.begin(), columns_names.end()); + /** The current user must have the SELECT privilege. + * We do not check access rights for table functions because they have beein already checked in ITablefunction::execute(). + */ + if (table_node) + checkAccessRights(*table_node, column_names, planner_context->getQueryContext()); + if (column_names.empty()) { auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); @@ -88,7 +128,6 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, table_expression_data.addColumn(additional_column_to_read, column_identifier); } - const auto & query_context = planner_context->getQueryContext(); size_t max_block_size = query_context->getSettingsRef().max_block_size; size_t max_streams = query_context->getSettingsRef().max_threads; From 98eba24a3495c99b57fbd4c825a1130bcb5380c6 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 12 Oct 2022 16:39:27 +0200 Subject: [PATCH 137/188] Fix LIMIT when query has WITH TOTALS --- src/Planner/Planner.cpp | 21 ++++++++++-- src/Planner/Utils.cpp | 76 ++++++++++++++++++++++++++++++++++++++--- src/Planner/Utils.h | 7 +++- 3 files changed, 95 insertions(+), 9 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 40761a34c5e..fe43dcf3dcc 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -658,13 +658,13 @@ void Planner::buildQueryPlanIfNeeded() { sort_description = extractSortDescription(query_node.getOrderByNode(), *planner_context); - bool query_has_array_join = queryHasArrayJoin(query_tree); + bool query_has_array_join_in_join_tree = queryHasArrayJoinInJoinTree(query_tree); UInt64 partial_sorting_limit = 0; /// Partial sort can be done if there is LIMIT, but no DISTINCT, LIMIT WITH TIES, LIMIT BY, ARRAY JOIN. - if (limit_length != 0 && !query_node.isDistinct() && !query_node.hasLimitBy() && !query_node.isLimitWithTies() && !query_has_array_join && - limit_length <= std::numeric_limits::max() - limit_offset) + if (limit_length != 0 && !query_node.isDistinct() && !query_node.hasLimitBy() && !query_node.isLimitWithTies() && + !query_has_array_join_in_join_tree && limit_length <= std::numeric_limits::max() - limit_offset) { partial_sorting_limit = limit_length + limit_offset; } @@ -816,6 +816,21 @@ void Planner::buildQueryPlanIfNeeded() bool always_read_till_end = settings.exact_rows_before_limit; bool limit_with_ties = query_node.isLimitWithTies(); + /** Special cases: + * + * 1. If there is WITH TOTALS and there is no ORDER BY, then read the data to the end, + * otherwise TOTALS is counted according to incomplete data. + * + * 2. If there is no WITH TOTALS and there is a subquery in FROM, and there is WITH TOTALS on one of the levels, + * then when using LIMIT, you should read the data to the end, rather than cancel the query earlier, + * because if you cancel the query, we will not get `totals` data from the remote server. + */ + if (query_node.isGroupByWithTotals() && !query_node.hasOrderBy()) + always_read_till_end = true; + + if (!query_node.isGroupByWithTotals() && queryHasWithTotalsInAnySubqueryInJoinTree(query_tree)) + always_read_till_end = true; + SortDescription limit_with_ties_sort_description; if (query_node.isLimitWithTies()) diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 3f4a532b462..9b18a3a0ab5 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -10,9 +10,11 @@ #include -#include #include #include +#include +#include +#include #include #include @@ -195,13 +197,12 @@ bool sortDescriptionIsPrefix(const SortDescription & prefix, const SortDescripti return true; } -bool queryHasArrayJoin(const QueryTreeNodePtr & query_node) +bool queryHasArrayJoinInJoinTree(const QueryTreeNodePtr & query_node) { const auto & query_node_typed = query_node->as(); - auto join_tree_node = query_node_typed.getJoinTree(); std::vector join_tree_nodes_to_process; - join_tree_nodes_to_process.push_back(std::move(join_tree_node)); + join_tree_nodes_to_process.push_back(query_node_typed.getJoinTree()); while (join_tree_nodes_to_process.empty()) { @@ -228,7 +229,72 @@ bool queryHasArrayJoin(const QueryTreeNodePtr & query_node) } case QueryTreeNodeType::JOIN: { - auto & join_node = join_tree_node->as(); + auto & join_node = join_tree_node_to_process->as(); + join_tree_nodes_to_process.push_back(join_node.getLeftTableExpression()); + join_tree_nodes_to_process.push_back(join_node.getRightTableExpression()); + break; + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected node type for table expression. Expected table, table function, query, union, join or array join. Actual {}", + join_tree_node_to_process->getNodeTypeName()); + } + } + } + + return false; +} + +bool queryHasWithTotalsInAnySubqueryInJoinTree(const QueryTreeNodePtr & query_node) +{ + const auto & query_node_typed = query_node->as(); + + std::vector join_tree_nodes_to_process; + join_tree_nodes_to_process.push_back(query_node_typed.getJoinTree()); + + while (join_tree_nodes_to_process.empty()) + { + auto join_tree_node_to_process = join_tree_nodes_to_process.back(); + join_tree_nodes_to_process.pop_back(); + + auto join_tree_node_type = join_tree_node_to_process->getNodeType(); + + switch (join_tree_node_type) + { + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + break; + } + case QueryTreeNodeType::QUERY: + { + auto & query_node_to_process = join_tree_node_to_process->as(); + if (query_node_to_process.isGroupByWithTotals()) + return true; + + join_tree_nodes_to_process.push_back(query_node_to_process.getJoinTree()); + break; + } + case QueryTreeNodeType::UNION: + { + auto & union_node = join_tree_node_to_process->as(); + auto & union_queries = union_node.getQueries().getNodes(); + + for (auto & union_query : union_queries) + join_tree_nodes_to_process.push_back(union_query); + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + auto & array_join_node = join_tree_node_to_process->as(); + join_tree_nodes_to_process.push_back(array_join_node.getTableExpression()); + break; + } + case QueryTreeNodeType::JOIN: + { + auto & join_node = join_tree_node_to_process->as(); join_tree_nodes_to_process.push_back(join_node.getLeftTableExpression()); join_tree_nodes_to_process.push_back(join_node.getRightTableExpression()); break; diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 5be2f75dd17..82026b1d72e 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -49,6 +49,11 @@ ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & express bool sortDescriptionIsPrefix(const SortDescription & prefix, const SortDescription & full); /// Returns true if query node JOIN TREE contains ARRAY JOIN node -bool queryHasArrayJoin(const QueryTreeNodePtr & query_node); +bool queryHasArrayJoinInJoinTree(const QueryTreeNodePtr & query_node); + +/** Returns true if query node JOIN TREE contains QUERY node with WITH TOTALS, false otherwise. + * Function is applied recursively to subqueries in JOIN TREE. + */ +bool queryHasWithTotalsInAnySubqueryInJoinTree(const QueryTreeNodePtr & query_node); } From 43ec5c830c641da7e9ade77d5f6a2992a0ac23d6 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 12 Oct 2022 18:09:39 +0200 Subject: [PATCH 138/188] Added FunctionToSubcolumnsPass --- .../Passes/FunctionToSubcolumnsPass.cpp | 234 ++++++++++++++++++ .../Passes/FunctionToSubcolumnsPass.h | 32 +++ src/Analyzer/Passes/QueryAnalysisPass.cpp | 8 +- src/Analyzer/Passes/QueryAnalysisPass.h | 2 +- src/Analyzer/QueryTreePassManager.cpp | 5 +- 5 files changed, 277 insertions(+), 4 deletions(-) create mode 100644 src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp create mode 100644 src/Analyzer/Passes/FunctionToSubcolumnsPass.h diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp new file mode 100644 index 00000000000..b6efcab811e --- /dev/null +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -0,0 +1,234 @@ +#include + +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace +{ + +class FunctionToSubcolumnsVisitor : public InDepthQueryTreeVisitor +{ +public: + explicit FunctionToSubcolumnsVisitor(ContextPtr & context_) + : context(context_) + {} + + void visitImpl(QueryTreeNodePtr & node) const + { + auto * function_node = node->as(); + if (!function_node) + return; + + auto function_arguments_nodes = function_node->getArguments().getNodes(); + size_t function_arguments_nodes_size = function_arguments_nodes.size(); + + if (function_arguments_nodes.empty() || function_arguments_nodes_size > 2) + return; + + auto * first_argument_column_node = function_arguments_nodes.front()->as(); + + if (!first_argument_column_node) + return; + + auto column_source = first_argument_column_node->getColumnSource(); + auto * table_node = column_source->as(); + + if (!table_node) + return; + + const auto & storage = table_node->getStorage(); + if (!storage->supportsSubcolumns()) + return; + + auto column = first_argument_column_node->getColumn(); + WhichDataType column_type(column.type); + + const auto & function_name = function_node->getFunctionName(); + + if (function_arguments_nodes_size == 1) + { + if (column_type.isArray()) + { + if (function_name == "length") + { + /// Replace `length(array_argument)` with `array_argument.length` + column.name += ".size0"; + + node = std::make_shared(column, column_source); + } + else if (function_name == "empty") + { + /// Replace `empty(array_argument)` with `equals(array_argument.size0, 0)` + column.name += ".size0"; + column.type = std::make_shared(); + + auto equals_function = std::make_shared("equals"); + resolveOrdinaryFunctionNode(*equals_function, "equals"); + + auto & equals_function_arguments = equals_function->getArguments().getNodes(); + equals_function_arguments.reserve(2); + equals_function_arguments.push_back(std::make_shared(column, column_source)); + equals_function_arguments.push_back(std::make_shared(static_cast(0))); + + node = std::move(equals_function); + } + else if (function_name == "notEmpty") + { + /// Replace `notEmpty(array_argument)` with `notEquals(array_argument.size0, 0)` + column.name += ".size0"; + column.type = std::make_shared(); + + auto not_equals_function = std::make_shared("notEquals"); + resolveOrdinaryFunctionNode(*not_equals_function, "notEquals"); + + auto & not_equals_function_arguments = not_equals_function->getArguments().getNodes(); + not_equals_function_arguments.reserve(2); + not_equals_function_arguments.push_back(std::make_shared(column, column_source)); + not_equals_function_arguments.push_back(std::make_shared(static_cast(0))); + + node = std::move(not_equals_function); + } + } + else if (column_type.isNullable()) + { + if (function_name == "isNull") + { + /// Replace `isNull(nullable_argument)` with `nullable_argument.null` + column.name += ".null"; + + node = std::make_shared(column, column_source); + } + else if (function_name == "isNotNull") + { + /// Replace `isNotNull(nullable_argument)` with `not(nullable_argument.null)` + column.name += ".null"; + column.type = std::make_shared(); + + auto not_function = std::make_shared("not"); + resolveOrdinaryFunctionNode(*not_function, "not"); + + auto & not_function_arguments = not_function->getArguments().getNodes(); + not_function_arguments.reserve(2); + not_function_arguments.push_back(std::make_shared(column, column_source)); + not_function_arguments.push_back(std::make_shared(static_cast(0))); + + node = std::move(not_function); + } + } + else if (column_type.isMap()) + { + const auto & data_type_map = assert_cast(*column.type); + + if (function_name == "mapKeys") + { + /// Replace `mapKeys(map_argument)` with `map_argument.keys` + column.name += ".keys"; + column.type = data_type_map.getKeyType(); + + node = std::make_shared(column, column_source); + } + else if (function_name == "mapValues") + { + /// Replace `mapValues(map_argument)` with `map_argument.values` + column.name += ".values"; + column.type = data_type_map.getValueType(); + + node = std::make_shared(column, column_source); + } + } + } + else + { + auto second_argument_constant_value = function_arguments_nodes[1]->getConstantValueOrNull(); + + if (function_name == "tupleElement" && column_type.isTuple() && second_argument_constant_value) + { + /** Replace `tupleElement(tuple_argument, string_literal)`, `tupleElement(tuple_argument, integer_literal)` + * with `tuple_argument.column_name`. + */ + const auto & tuple_element_constant_value = second_argument_constant_value->getValue(); + const auto & tuple_element_constant_value_type = tuple_element_constant_value.getType(); + + const auto & data_type_tuple = assert_cast(*column.type); + + String subcolumn_name; + + if (tuple_element_constant_value_type == Field::Types::String) + { + subcolumn_name = tuple_element_constant_value.get(); + } + else if (tuple_element_constant_value_type == Field::Types::UInt64) + { + auto tuple_column_index = tuple_element_constant_value.get(); + subcolumn_name = data_type_tuple.getNameByPosition(tuple_column_index); + } + else + { + return; + } + + column.name += "."; + column.name += subcolumn_name; + + size_t subcolumn_position = data_type_tuple.getPositionByName(subcolumn_name); + column.type = data_type_tuple.getElement(subcolumn_position); + + node = std::make_shared(column, column_source); + } + else if (function_name == "mapContains" && column_type.isMap()) + { + const auto & data_type_map = assert_cast(*column.type); + + /// Replace `mapContains(map_argument, argument)` with `has(map_argument.keys, argument)` + column.name += ".keys"; + column.type = data_type_map.getKeyType(); + + auto has_function_argument = std::make_shared(column, column_source); + auto has_function = std::make_shared("has"); + resolveOrdinaryFunctionNode(*has_function, "has"); + + auto & has_function_arguments = has_function->getArguments().getNodes(); + has_function_arguments.reserve(2); + has_function_arguments.push_back(std::move(has_function_argument)); + has_function_arguments.push_back(std::move(function_arguments_nodes[1])); + + node = std::move(has_function); + } + } + } + + inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const + { + auto function_result_type = function_node.getResultType(); + auto function = FunctionFactory::instance().get(function_name, context); + function_node.resolveAsFunction(function, std::move(function_result_type)); + } + +private: + ContextPtr & context; +}; + +} + +void FunctionToSubcolumnsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + FunctionToSubcolumnsVisitor visitor(context); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.h b/src/Analyzer/Passes/FunctionToSubcolumnsPass.h new file mode 100644 index 00000000000..d5c1dc3805d --- /dev/null +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.h @@ -0,0 +1,32 @@ +#pragma once + +#include + +namespace DB +{ + +/** Transform functions to subcolumns. + * It can help to reduce amount of read data. + * + * Example: SELECT tupleElement(column, subcolumn) FROM test_table; + * Result: SELECT column.subcolumn FROM test_table; + * + * Example: SELECT length(array_column) FROM test_table; + * Result: SELECT array_column.size0 FROM test_table; + * + * Example: SELECT nullable_column IS NULL FROM test_table; + * Result: SELECT nullable_column.null FROM test_table; + */ +class FunctionToSubcolumnsPass final : public IQueryTreePass +{ +public: + String getName() override { return "FunctionToSubcolumnsPass"; } + + String getDescription() override { return "Rewrite function to subcolumns, for example tupleElement(column, subcolumn) into column.subcolumn"; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} + diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 86ffc9b7d44..2ad9b84ef7c 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5524,12 +5524,16 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /** Validate aggregates * - * 1. Check that there are no aggregate functions and GROUPING function in WHERE, in PREWHERE, in another aggregate functions. - * 2. Check that there are no window functions in WHERE, in PREWHERE, in HAVING, in WINDOW, inside another aggregate function, + * 1. Check that there are no aggregate functions and GROUPING function in JOIN TREE, WHERE, PREWHERE, in another aggregate functions. + * 2. Check that there are no window functions in JOIN TREE, WHERE, PREWHERE, HAVING, WINDOW, inside another aggregate function, * inside window function arguments, inside window function window definition. * 3. Check that there are no columns that are not specified in GROUP BY keys. * 4. Validate GROUP BY modifiers. */ + assertNoAggregateFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); + assertNoGroupingFunction(query_node_typed.getJoinTree(), "in JOIN TREE"); + assertNoWindowFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); + if (query_node_typed.hasWhere()) { assertNoAggregateFunctionNodes(query_node_typed.getWhere(), "in WHERE"); diff --git a/src/Analyzer/Passes/QueryAnalysisPass.h b/src/Analyzer/Passes/QueryAnalysisPass.h index 231485d4bca..3ae2713a2ee 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.h +++ b/src/Analyzer/Passes/QueryAnalysisPass.h @@ -37,7 +37,7 @@ namespace DB * In SELECT, ORDER BY only columns that are specified in GROUP BY keys after GROUP BY are used. * GROUPING function arguments are specified in GROUP BY keys. * No GROUPING function if there is no GROUP BY. - * No aggregate functions in WHERE, PREWHERE, GROUP BY and inside another aggregate functions. + * No aggregate functions in JOIN TREE, WHERE, PREWHERE, GROUP BY and inside another aggregate functions. * GROUP BY modifiers CUBE, ROLLUP, GROUPING SETS and WITH TOTALS. * Table expression modifiers are disabled for subqueries in JOIN TREE. * For JOIN, ARRAY JOIN subqueries and table functions must have alias (Can be changed using joined_subquery_requires_alias setting). diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index b5146b65f6f..eb6c5ac34a9 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -28,7 +29,6 @@ namespace ErrorCodes * TODO: Support _shard_num into shardNum() rewriting. * TODO: Support logical expressions optimizer. * TODO: Support fuse sum count optimize_fuse_sum_count_avg, optimize_syntax_fuse_functions. - * TODO: Support setting optimize_functions_to_subcolumns. * TODO: Support setting optimize_arithmetic_operations_in_aggregate_functions. * TODO: Support setting convert_query_to_cnf. * TODO: Support setting optimize_using_constraints. @@ -115,6 +115,9 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_shared()); + if (settings.optimize_functions_to_subcolumns) + manager.addPass(std::make_shared()); + if (settings.count_distinct_optimization) manager.addPass(std::make_shared()); From fe3eda4c2b9422ac84a98205ab185261ba1a7988 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 17 Oct 2022 15:51:09 +0200 Subject: [PATCH 139/188] Added SumIfToCountIfPass --- .../Passes/FunctionToSubcolumnsPass.cpp | 13 +- src/Analyzer/Passes/IfChainToMultiIfPass.cpp | 2 +- src/Analyzer/Passes/SumIfToCountIfPass.cpp | 156 ++++++++++++++++++ src/Analyzer/Passes/SumIfToCountIfPass.h | 31 ++++ src/Analyzer/QueryTreePassManager.cpp | 8 +- 5 files changed, 200 insertions(+), 10 deletions(-) create mode 100644 src/Analyzer/Passes/SumIfToCountIfPass.cpp create mode 100644 src/Analyzer/Passes/SumIfToCountIfPass.h diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index b6efcab811e..a112d2e6886 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -6,14 +6,15 @@ #include +#include + +#include + #include #include #include #include #include -#include - -#include namespace DB { @@ -123,9 +124,7 @@ public: resolveOrdinaryFunctionNode(*not_function, "not"); auto & not_function_arguments = not_function->getArguments().getNodes(); - not_function_arguments.reserve(2); not_function_arguments.push_back(std::make_shared(column, column_source)); - not_function_arguments.push_back(std::make_shared(static_cast(0))); node = std::move(not_function); } @@ -182,7 +181,7 @@ public: return; } - column.name += "."; + column.name += '.'; column.name += subcolumn_name; size_t subcolumn_position = data_type_tuple.getPositionByName(subcolumn_name); @@ -212,6 +211,7 @@ public: } } +private: inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const { auto function_result_type = function_node.getResultType(); @@ -219,7 +219,6 @@ public: function_node.resolveAsFunction(function, std::move(function_result_type)); } -private: ContextPtr & context; }; diff --git a/src/Analyzer/Passes/IfChainToMultiIfPass.cpp b/src/Analyzer/Passes/IfChainToMultiIfPass.cpp index 3fb773b070b..f400b11765e 100644 --- a/src/Analyzer/Passes/IfChainToMultiIfPass.cpp +++ b/src/Analyzer/Passes/IfChainToMultiIfPass.cpp @@ -19,7 +19,7 @@ public: : multi_if_function_ptr(std::move(multi_if_function_ptr_)) {} - void visit(QueryTreeNodePtr & node) + void visitImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || function_node->getFunctionName() != "if" || function_node->getArguments().getNodes().size() != 3) diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.cpp b/src/Analyzer/Passes/SumIfToCountIfPass.cpp new file mode 100644 index 00000000000..8dde6d9f4d4 --- /dev/null +++ b/src/Analyzer/Passes/SumIfToCountIfPass.cpp @@ -0,0 +1,156 @@ +#include + +#include +#include + +#include + +#include + +#include +#include + +namespace DB +{ + +namespace +{ + +class SumIfToCountIfVisitor : public InDepthQueryTreeVisitor +{ +public: + explicit SumIfToCountIfVisitor(ContextPtr & context_) + : context(context_) + {} + + void visitImpl(QueryTreeNodePtr & node) + { + auto * function_node = node->as(); + if (!function_node || !function_node->isAggregateFunction()) + return; + + auto function_name = function_node->getFunctionName(); + auto lower_function_name = Poco::toLower(function_name); + + /// sumIf, SumIf or sUMIf are valid function names, but sumIF or sumiF are not + if (lower_function_name != "sum" && (lower_function_name != "sumif" || !function_name.ends_with("If"))) + return; + + auto & function_node_arguments_nodes = function_node->getArguments().getNodes(); + + /// Rewrite `sumIf(1, cond)` into `countIf(cond)` + if (lower_function_name == "sumif") + { + if (function_node_arguments_nodes.size() != 2) + return; + + auto constant_value = function_node_arguments_nodes[0]->getConstantValueOrNull(); + if (!constant_value) + return; + + const auto & constant_value_literal = constant_value->getValue(); + if (!isInt64OrUInt64FieldType(constant_value_literal.getType())) + return; + + if (constant_value_literal.get() != 1) + return; + + function_node_arguments_nodes[0] = std::move(function_node_arguments_nodes[1]); + function_node_arguments_nodes.resize(1); + + resolveAggregateFunctionNode(*function_node, "countIf"); + return; + } + + /** Rewrite `sum(if(cond, 1, 0))` into `countIf(cond)`. + * Rewrite `sum(if(cond, 0, 1))` into `countIf(not(cond))`. + */ + if (function_node_arguments_nodes.size() != 1) + return; + + auto & nested_argument = function_node_arguments_nodes[0]; + auto * nested_function = nested_argument->as(); + if (!nested_function || nested_function->getFunctionName() != "if") + return; + + auto nested_if_function_arguments_nodes = nested_function->getArguments().getNodes(); + if (nested_if_function_arguments_nodes.size() != 3) + return; + + auto if_true_condition_constant_value = nested_if_function_arguments_nodes[1]->getConstantValueOrNull(); + auto if_false_condition_constant_value = nested_if_function_arguments_nodes[2]->getConstantValueOrNull(); + + if (!if_true_condition_constant_value || !if_false_condition_constant_value) + return; + + const auto & if_true_condition_constant_value_literal = if_true_condition_constant_value->getValue(); + const auto & if_false_condition_constant_value_literal = if_false_condition_constant_value->getValue(); + + if (!isInt64OrUInt64FieldType(if_true_condition_constant_value_literal.getType()) || + !isInt64OrUInt64FieldType(if_false_condition_constant_value_literal.getType())) + return; + + auto if_true_condition_value = if_true_condition_constant_value_literal.get(); + auto if_false_condition_value = if_false_condition_constant_value_literal.get(); + + /// Rewrite `sum(if(cond, 1, 0))` into `countIf(cond)`. + if (if_true_condition_value == 1 && if_false_condition_value == 0) + { + function_node_arguments_nodes[0] = std::move(nested_if_function_arguments_nodes[0]); + function_node_arguments_nodes.resize(1); + + resolveAggregateFunctionNode(*function_node, "countIf"); + return; + } + + /// Rewrite `sum(if(cond, 0, 1))` into `countIf(not(cond))`. + if (if_true_condition_value == 0 && if_false_condition_value == 1) + { + auto not_function = std::make_shared("not"); + resolveOrdinaryFunctionNode(*not_function, "not"); + + auto & not_function_arguments = not_function->getArguments().getNodes(); + not_function_arguments.push_back(std::move(nested_if_function_arguments_nodes[0])); + + function_node_arguments_nodes[0] = std::move(not_function); + function_node_arguments_nodes.resize(1); + + resolveAggregateFunctionNode(*function_node, "countIf"); + return; + } + } + +private: + inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const + { + auto function_result_type = function_node.getResultType(); + auto function = FunctionFactory::instance().get(function_name, context); + function_node.resolveAsFunction(function, std::move(function_result_type)); + } + + static inline void resolveAggregateFunctionNode(FunctionNode & function_node, const String & aggregate_function_name) + { + auto function_result_type = function_node.getResultType(); + auto function_aggregate_function = function_node.getAggregateFunction(); + + AggregateFunctionProperties properties; + auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, + function_aggregate_function->getArgumentTypes(), + function_aggregate_function->getParameters(), + properties); + + function_node.resolveAsAggregateFunction(std::move(aggregate_function), std::move(function_result_type)); + } + + ContextPtr & context; +}; + +} + +void SumIfToCountIfPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + SumIfToCountIfVisitor visitor(context); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.h b/src/Analyzer/Passes/SumIfToCountIfPass.h new file mode 100644 index 00000000000..4f5f189d4a3 --- /dev/null +++ b/src/Analyzer/Passes/SumIfToCountIfPass.h @@ -0,0 +1,31 @@ +#pragma once + +#include + +namespace DB +{ + +/** Rewrite `sum(if(cond, value_1, value_2))` and `sumIf` functions to `countIf`. + * + * Example: SELECT sumIf(1, cond); + * Result: SELECT countIf(cond); + * + * Example: SELECT sum(if(cond, 1, 0)); + * Result: SELECT countIf(cond); + * + * Example: SELECT sum(if(cond, 0, 1)); + * Result: SELECT countIf(not(cond)); + */ +class SumIfToCountIfPass final : public IQueryTreePass +{ +public: + String getName() override { return "SumIfToCountIfPass"; } + + String getDescription() override { return "Rewrite sum(if) and sumIf into countIf"; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} + diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index eb6c5ac34a9..6b8608a87ef 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -1,13 +1,14 @@ #include #include +#include #include +#include #include #include #include #include #include -#include #include #include @@ -36,7 +37,7 @@ namespace ErrorCodes * TODO: Support GROUP BY injective function elimination. * TODO: Support GROUP BY functions of other keys elimination. * TODO: Support setting optimize_move_functions_out_of_any. - * TODO: Support setting optimize_rewrite_sum_if_to_count_if. + * TODO: Support setting optimize_injective_functions_inside_uniq. * TODO: Support setting optimize_aggregators_of_group_by_keys. * TODO: Support setting optimize_duplicate_order_by_and_distinct. * TODO: Support setting optimize_redundant_functions_in_order_by. @@ -121,6 +122,9 @@ void addQueryTreePasses(QueryTreePassManager & manager) if (settings.count_distinct_optimization) manager.addPass(std::make_shared()); + if (settings.optimize_rewrite_sum_if_to_count_if) + manager.addPass(std::make_shared()); + if (settings.optimize_normalize_count_variants) manager.addPass(std::make_shared()); From c82bb1128fcec3d1fd95c1073bed5a318a3c3788 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 17 Oct 2022 17:09:45 +0200 Subject: [PATCH 140/188] Fixed nested IN functions --- src/Planner/CollectSets.cpp | 20 ++-- src/Planner/CollectSets.h | 4 +- src/Planner/Planner.cpp | 16 +-- src/Planner/PlannerActionsVisitor.cpp | 11 +- src/Planner/PlannerContext.cpp | 97 ++++++++-------- src/Planner/PlannerContext.h | 106 +++++++++++------- ...er_in_function_nested_subqueries.reference | 1 + ...analyzer_in_function_nested_subqueries.sql | 3 + 8 files changed, 145 insertions(+), 113 deletions(-) create mode 100644 tests/queries/0_stateless/02386_analyzer_in_function_nested_subqueries.reference create mode 100644 tests/queries/0_stateless/02386_analyzer_in_function_nested_subqueries.sql diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 65348c38d4a..aa7014aba48 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -25,7 +25,7 @@ namespace class CollectSetsVisitor : public ConstInDepthQueryTreeVisitor { public: - explicit CollectSetsVisitor(const PlannerContext & planner_context_) + explicit CollectSetsVisitor(PlannerContext & planner_context_) : planner_context(planner_context_) {} @@ -39,13 +39,11 @@ public: auto in_second_argument = function_node->getArguments().getNodes().at(1); auto in_second_argument_node_type = in_second_argument->getNodeType(); - const auto & global_planner_context = planner_context.getGlobalPlannerContext(); const auto & settings = planner_context.getQueryContext()->getSettingsRef(); - String set_key = global_planner_context->createSetKey(in_second_argument); - auto prepared_set = global_planner_context->getSetOrNull(set_key); + String set_key = planner_context.createSetKey(in_second_argument); - if (prepared_set) + if (planner_context.hasSet(set_key)) return; /// Tables and table functions are replaced with subquery at Analysis stage, except special Set table. @@ -54,7 +52,7 @@ public: if (storage_set) { - global_planner_context->registerSet(set_key, storage_set->getSet()); + planner_context.registerSet(set_key, PlannerSet(storage_set->getSet())); } else if (auto constant_value = in_second_argument->getConstantValueOrNull()) { @@ -64,18 +62,16 @@ public: constant_value->getType(), settings); - global_planner_context->registerSet(set_key, std::move(set)); + planner_context.registerSet(set_key, PlannerSet(std::move(set))); } else if (in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION) { SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; bool tranform_null_in = settings.transform_null_in; - auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); - global_planner_context->registerSet(set_key, set); - global_planner_context->registerSubqueryNodeForSet(set_key, SubqueryNodeForSet{in_second_argument, set}); + planner_context.registerSet(set_key, PlannerSet(std::move(set), in_second_argument)); } else { @@ -91,12 +87,12 @@ public: } private: - const PlannerContext & planner_context; + PlannerContext & planner_context; }; } -void collectSets(const QueryTreeNodePtr & node, const PlannerContext & planner_context) +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context) { CollectSetsVisitor visitor(planner_context); visitor.visit(node); diff --git a/src/Planner/CollectSets.h b/src/Planner/CollectSets.h index a641d12f9d9..94f792e877b 100644 --- a/src/Planner/CollectSets.h +++ b/src/Planner/CollectSets.h @@ -8,8 +8,8 @@ namespace DB { /** Collect prepared sets and sets for subqueries that are necessary to execute IN function and its variations. - * Collected sets are registered in global planner context. + * Collected sets are registered in planner context. */ -void collectSets(const QueryTreeNodePtr & node, const PlannerContext & planner_context); +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index fe43dcf3dcc..ca991b76c94 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -133,25 +133,27 @@ void checkStoragesSupportTransactions(const PlannerContextPtr & planner_context) void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, const SelectQueryOptions & select_query_options, const PlannerContextPtr & planner_context) { - if (select_query_options.is_subquery) - return; - PreparedSets::SubqueriesForSets subqueries_for_sets; - const auto & subquery_node_to_sets = planner_context->getGlobalPlannerContext()->getSubqueryNodesForSets(); + const auto & set_key_to_planner_set = planner_context->getRegisteredSets(); - for (auto [key, subquery_node_for_set] : subquery_node_to_sets) + for (auto [key, planner_set] : set_key_to_planner_set) { + const auto subquery_node = planner_set.getSubqueryNode(); + if (!subquery_node) + continue; + auto subquery_context = buildSubqueryContext(planner_context->getQueryContext()); auto subquery_options = select_query_options.subquery(); + Planner subquery_planner( - subquery_node_for_set.subquery_node, + subquery_node, subquery_options, std::move(subquery_context), planner_context->getGlobalPlannerContext()); subquery_planner.buildQueryPlanIfNeeded(); SubqueryForSet subquery_for_set; - subquery_for_set.set = subquery_node_for_set.set; + subquery_for_set.set = planner_set.getSet(); subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); subqueries_for_sets.emplace(key, std::move(subquery_for_set)); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index fb1badea2dd..10484eef454 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -349,16 +349,15 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma const auto & function_node = node->as(); auto in_second_argument = function_node.getArguments().getNodes().at(1); - const auto & global_planner_context = planner_context->getGlobalPlannerContext(); - auto set_key = global_planner_context->createSetKey(in_second_argument); - auto prepared_set = global_planner_context->getSetOrThrow(set_key); + auto set_key = planner_context->createSetKey(in_second_argument); + auto planner_set = planner_context->getSetOrThrow(set_key); ColumnWithTypeAndName column; column.name = set_key; column.type = std::make_shared(); - bool set_is_created = prepared_set->isCreated(); - auto column_set = ColumnSet::create(1, std::move(prepared_set)); + bool set_is_created = planner_set.getSet()->isCreated(); + auto column_set = ColumnSet::create(1, planner_set.getSet()); if (set_is_created) column.column = ColumnConst::create(std::move(column_set), 1); @@ -527,7 +526,7 @@ String calculateActionNodeName(const QueryTreeNodePtr & node, const PlannerConte if (isNameOfInFunction(function_node.getFunctionName())) { const auto & in_second_argument_node = function_node.getArguments().getNodes().at(1); - in_function_second_argument_node_name = planner_context.getGlobalPlannerContext()->createSetKey(in_second_argument_node); + in_function_second_argument_node_name = planner_context.createSetKey(in_second_argument_node); } WriteBufferFromOwnString buffer; diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index f2be620042e..9f4a489bf5f 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -11,52 +11,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -GlobalPlannerContext::SetKey GlobalPlannerContext::createSetKey(const QueryTreeNodePtr & set_source_node) -{ - auto set_source_hash = set_source_node->getTreeHash(); - return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); -} - -void GlobalPlannerContext::registerSet(const SetKey & key, SetPtr set) -{ - set_key_to_set.emplace(key, std::move(set)); -} - -SetPtr GlobalPlannerContext::getSetOrNull(const SetKey & key) const -{ - auto it = set_key_to_set.find(key); - if (it == set_key_to_set.end()) - return nullptr; - - return it->second; -} - -SetPtr GlobalPlannerContext::getSetOrThrow(const SetKey & key) const -{ - auto it = set_key_to_set.find(key); - if (it == set_key_to_set.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "No set is registered for key {}", - key); - - return it->second; -} - -void GlobalPlannerContext::registerSubqueryNodeForSet(const SetKey & key, SubqueryNodeForSet subquery_node_for_set) -{ - auto node_type = subquery_node_for_set.subquery_node->getNodeType(); - if (node_type != QueryTreeNodeType::QUERY && - node_type != QueryTreeNodeType::UNION) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Invalid node for set table expression. Expected query or union. Actual {}", - subquery_node_for_set.subquery_node->formatASTForErrorMessage()); - if (!subquery_node_for_set.set) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Set must be initialized"); - - set_key_to_subquery_node.emplace(key, std::move(subquery_node_for_set)); -} - const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const QueryTreeNodePtr & column_node) { const auto & column_node_typed = column_node->as(); @@ -166,4 +120,55 @@ const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const Que return table_expression_data->getColumnIdentifierOrNull(column_name); } +PlannerContext::SetKey PlannerContext::createSetKey(const QueryTreeNodePtr & set_source_node) +{ + auto set_source_hash = set_source_node->getTreeHash(); + return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); +} + +void PlannerContext::registerSet(const SetKey & key, PlannerSet planner_set) +{ + if (!planner_set.getSet()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Set must be initialized"); + + const auto & subquery_node = planner_set.getSubqueryNode(); + if (subquery_node) + { + auto node_type = subquery_node->getNodeType(); + + if (node_type != QueryTreeNodeType::QUERY && + node_type != QueryTreeNodeType::UNION) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Invalid node for set table expression. Expected query or union. Actual {}", + subquery_node->formatASTForErrorMessage()); + } + + set_key_to_set.emplace(key, std::move(planner_set)); +} + +bool PlannerContext::hasSet(const SetKey & key) const +{ + return set_key_to_set.contains(key); +} + +const PlannerSet & PlannerContext::getSetOrThrow(const SetKey & key) const +{ + auto it = set_key_to_set.find(key); + if (it == set_key_to_set.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "No set is registered for key {}", + key); + + return it->second; +} + +const PlannerSet * PlannerContext::getSetOrNull(const SetKey & key) const +{ + auto it = set_key_to_set.find(key); + if (it == set_key_to_set.end()) + return nullptr; + + return &it->second; +} + } diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 22c746c82c9..b4f530cae65 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -15,50 +15,15 @@ namespace DB { -/// Subquery node for set -struct SubqueryNodeForSet -{ - QueryTreeNodePtr subquery_node; - SetPtr set; -}; - /** Global planner context contains common objects that are shared between each planner context. * - * 1. Prepared sets. - * 2. Subqueries for sets. + * 1. Column identifiers. */ class GlobalPlannerContext { public: GlobalPlannerContext() = default; - using SetKey = std::string; - using SetKeyToSet = std::unordered_map; - using SetKeyToSubqueryNode = std::unordered_map; - - /// Create set key for query node - static SetKey createSetKey(const QueryTreeNodePtr & set_source_node); - - /// Register set for set key - void registerSet(const SetKey & key, SetPtr set); - - /// Get set for key, if no set is registered null is returned - SetPtr getSetOrNull(const SetKey & key) const; - - /// Get set for key, if no set is registered logical exception is thrown - SetPtr getSetOrThrow(const SetKey & key) const; - - /** Register subquery node for set. - * Subquery node for set node must have QUERY or UNION type and set must be initialized. - */ - void registerSubqueryNodeForSet(const SetKey & key, SubqueryNodeForSet subquery_node_for_set); - - /// Get subquery nodes for sets - const SetKeyToSubqueryNode & getSubqueryNodesForSets() const - { - return set_key_to_subquery_node; - } - /** Create column identifier for column node. * * Result column identifier is added into context. @@ -75,15 +40,48 @@ public: bool hasColumnIdentifier(const ColumnIdentifier & column_identifier); private: - SetKeyToSet set_key_to_set; - - SetKeyToSubqueryNode set_key_to_subquery_node; - std::unordered_set column_identifiers; }; using GlobalPlannerContextPtr = std::shared_ptr; +/** PlannerSet is wrapper around Set that is used during query planning. + * + * If subquery node is null, such set is already prepared for execution. + * + * If subquery node is not null, then set must be build from the result of the subquery. + * If subquery node is not null, it must have QUERY or UNION type. + */ +class PlannerSet +{ +public: + /// Construct planner set that is ready for execution + explicit PlannerSet(SetPtr set_) + : set(std::move(set_)) + {} + + /// Construct planner set with set and subquery node + explicit PlannerSet(SetPtr set_, QueryTreeNodePtr subquery_node_) + : set(std::move(set_)) + , subquery_node(std::move(subquery_node_)) + {} + + const SetPtr & getSet() const + { + return set; + } + + const QueryTreeNodePtr & getSubqueryNode() const + { + return subquery_node; + } + +private: + SetPtr set; + + QueryTreeNodePtr subquery_node; +}; + class PlannerContext { public: @@ -157,6 +155,31 @@ public: */ const ColumnIdentifier * getColumnNodeIdentifierOrNull(const QueryTreeNodePtr & column_node) const; + using SetKey = std::string; + + using SetKeyToSet = std::unordered_map; + + /// Create set key for set source node + static SetKey createSetKey(const QueryTreeNodePtr & set_source_node); + + /// Register set for set key + void registerSet(const SetKey & key, PlannerSet planner_set); + + /// Returns true if set is registered for key, false otherwise + bool hasSet(const SetKey & key) const; + + /// Get set for key, if no set is registered logical exception is thrown + const PlannerSet & getSetOrThrow(const SetKey & key) const; + + /// Get set for key, if no set is registered null is returned + const PlannerSet * getSetOrNull(const SetKey & key) const; + + /// Get registered sets + const SetKeyToSet & getRegisteredSets() const + { + return set_key_to_set; + } + private: /// Query context ContextPtr query_context; @@ -170,6 +193,9 @@ private: /// Table expression node to data std::unordered_map table_expression_node_to_data; + /// Set key to set + SetKeyToSet set_key_to_set; + }; using PlannerContextPtr = std::shared_ptr; diff --git a/tests/queries/0_stateless/02386_analyzer_in_function_nested_subqueries.reference b/tests/queries/0_stateless/02386_analyzer_in_function_nested_subqueries.reference new file mode 100644 index 00000000000..dec7d2fabd2 --- /dev/null +++ b/tests/queries/0_stateless/02386_analyzer_in_function_nested_subqueries.reference @@ -0,0 +1 @@ +\N diff --git a/tests/queries/0_stateless/02386_analyzer_in_function_nested_subqueries.sql b/tests/queries/0_stateless/02386_analyzer_in_function_nested_subqueries.sql new file mode 100644 index 00000000000..bffbc46809a --- /dev/null +++ b/tests/queries/0_stateless/02386_analyzer_in_function_nested_subqueries.sql @@ -0,0 +1,3 @@ +SET use_analyzer = 1; + +SELECT (NULL IN (SELECT 9223372036854775806 IN (SELECT 65536), inf, NULL IN (NULL))) IN (SELECT NULL IN (NULL)); From 8610feb06dff1ba96a4eeb455b6994dbcc06243d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 17 Oct 2022 19:26:30 +0200 Subject: [PATCH 141/188] Added UniqInjectiveFunctionsEliminationPass --- .../UniqInjectiveFunctionsEliminationPass.cpp | 64 +++++++++++++++++++ .../UniqInjectiveFunctionsEliminationPass.h | 25 ++++++++ src/Analyzer/QueryTreePassManager.cpp | 5 +- 3 files changed, 93 insertions(+), 1 deletion(-) create mode 100644 src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp create mode 100644 src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h diff --git a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp new file mode 100644 index 00000000000..ed7b6d5c445 --- /dev/null +++ b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp @@ -0,0 +1,64 @@ +#include + +#include + +#include +#include + +namespace DB +{ + +namespace +{ + +bool isUniqFunction(const String & function_name) +{ + return function_name == "uniq" || + function_name == "uniqExact" || + function_name == "uniqHLL12" || + function_name == "uniqCombined" || + function_name == "uniqCombined64" || + function_name == "uniqTheta"; +} + +class UniqInjectiveFunctionsEliminationVisitor : public InDepthQueryTreeVisitor +{ +public: + void visitImpl(QueryTreeNodePtr & node) + { + auto * function_node = node->as(); + if (!function_node || !function_node->isAggregateFunction() || !isUniqFunction(function_node->getFunctionName())) + return; + + auto & uniq_function_arguments_nodes = function_node->getArguments().getNodes(); + for (auto & uniq_function_argument_node : uniq_function_arguments_nodes) + { + auto * uniq_function_argument_node_typed = uniq_function_argument_node->as(); + if (!uniq_function_argument_node_typed || !uniq_function_argument_node_typed->isOrdinaryFunction()) + continue; + + auto & uniq_function_argument_node_argument_nodes = uniq_function_argument_node_typed->getArguments().getNodes(); + + /// Do not apply optimization if injective function contains multiple arguments + if (uniq_function_argument_node_argument_nodes.size() != 1) + continue; + + const auto & uniq_function_argument_node_function = uniq_function_argument_node_typed->getFunction(); + if (!uniq_function_argument_node_function->isInjective({})) + continue; + + /// Replace injective function with its single argument + uniq_function_argument_node = std::move(uniq_function_argument_node_argument_nodes[0]); + } + } +}; + +} + +void UniqInjectiveFunctionsEliminationPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) +{ + UniqInjectiveFunctionsEliminationVisitor visitor; + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h new file mode 100644 index 00000000000..47fd21b26ba --- /dev/null +++ b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h @@ -0,0 +1,25 @@ +#pragma once + +#include + +namespace DB +{ + +/** Remove injective functions from `uniq*` functions arguments. + * + * Example: SELECT uniq(injectiveFunction(argument)) + * Result: SELECT uniq(argument); + */ +class UniqInjectiveFunctionsEliminationPass final : public IQueryTreePass +{ +public: + String getName() override { return "UniqInjectiveFunctionsEliminationPass"; } + + String getDescription() override { return "Remove injective functions from uniq functions arguments"; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} + diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 6b8608a87ef..539bfcd76f1 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -37,7 +38,6 @@ namespace ErrorCodes * TODO: Support GROUP BY injective function elimination. * TODO: Support GROUP BY functions of other keys elimination. * TODO: Support setting optimize_move_functions_out_of_any. - * TODO: Support setting optimize_injective_functions_inside_uniq. * TODO: Support setting optimize_aggregators_of_group_by_keys. * TODO: Support setting optimize_duplicate_order_by_and_distinct. * TODO: Support setting optimize_redundant_functions_in_order_by. @@ -130,6 +130,9 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_shared()); + if (settings.optimize_injective_functions_inside_uniq) + manager.addPass(std::make_shared()); + if (settings.optimize_multiif_to_if) manager.addPass(std::make_shared()); From 852dd8c2bbe2e3d72e42bb786c2c57b88455484d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 17 Oct 2022 19:46:30 +0200 Subject: [PATCH 142/188] Updated UnionNode documentation --- src/Analyzer/QueryNode.h | 2 +- src/Analyzer/TableNode.h | 4 ---- src/Analyzer/UnionNode.cpp | 6 +++-- src/Analyzer/UnionNode.h | 46 ++++++++++++++++++++++++++++---------- 4 files changed, 39 insertions(+), 19 deletions(-) diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index e93b8604a4d..6ab9e5a72bd 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -74,7 +74,7 @@ public: return is_subquery; } - /// Set query node is subquery + /// Set query node is subquery value void setIsSubquery(bool is_subquery_value) { is_subquery = is_subquery_value; diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index e154f62614a..54064428bff 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -16,10 +16,6 @@ namespace DB /** Table node represents table in query tree. * Example: SELECT a FROM test_table. * test_table - is identifier, that during query analysis pass must be resolved into table node. - * - * During construction table node: - * 1. Lock storage for share. Later lock can be moved out of node using `moveTableLock` method. - * 2. Take storage snapshot. */ class TableNode; using TableNodePtr = std::shared_ptr; diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 4cbda0f57cf..703c68316a7 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -41,10 +41,12 @@ UnionNode::UnionNode() NamesAndTypes UnionNode::computeProjectionColumns() const { std::vector projections; - const auto & query_nodes = getQueries().getNodes(); NamesAndTypes query_node_projection; + const auto & query_nodes = getQueries().getNodes(); + projections.reserve(query_nodes.size()); + for (const auto & query_node : query_nodes) { if (auto * query_node_typed = query_node->as()) @@ -234,7 +236,7 @@ ASTPtr UnionNode::toASTImpl() const if (union_mode != SelectUnionMode::UNION_DEFAULT && union_mode != SelectUnionMode::EXCEPT_DEFAULT && - union_mode != SelectUnionMode::EXCEPT_DEFAULT) + union_mode != SelectUnionMode::INTERSECT_DEFAULT) select_with_union_query->is_normalized = true; select_with_union_query->list_of_modes = union_modes; diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h index 43d5f6bd345..8fe14871fdd 100644 --- a/src/Analyzer/UnionNode.h +++ b/src/Analyzer/UnionNode.h @@ -19,6 +19,15 @@ namespace ErrorCodes } /** Union node represents union of queries in query tree. + * + * Example: (SELECT id FROM test_table) UNION ALL (SELECT id FROM test_table_2); + * Example: (SELECT id FROM test_table) UNION DISTINCT (SELECT id FROM test_table_2); + * Example: (SELECT id FROM test_table) EXCEPT ALL (SELECT id FROM test_table_2); + * Example: (SELECT id FROM test_table) EXCEPT DISTINCT (SELECT id FROM test_table_2); + * Example: (SELECT id FROM test_table) INTERSECT ALL (SELECT id FROM test_table_2); + * Example: (SELECT id FROM test_table) INTERSECT DISTINCT (SELECT id FROM test_table_2); + * + * During query analysis pass union node queries must be resolved. */ class UnionNode; using UnionNodePtr = std::shared_ptr; @@ -28,77 +37,91 @@ class UnionNode final : public IQueryTreeNode public: explicit UnionNode(); + /// Returns true if union node is subquery, false otherwise bool isSubquery() const { return is_subquery; } + /// Set union node is subquery value void setIsSubquery(bool is_subquery_value) { is_subquery = is_subquery_value; } + /// Returns true if union node is CTE, false otherwise bool isCTE() const { return is_cte; } + /// Set union node is CTE void setIsCTE(bool is_cte_value) { is_cte = is_cte_value; } + /// Get union node CTE name const std::string & getCTEName() const { return cte_name; } + /// Set union node CTE name void setCTEName(std::string cte_name_value) { cte_name = std::move(cte_name_value); } + /// Get union mode SelectUnionMode getUnionMode() const { return union_mode; } + /// Set union mode value void setUnionMode(SelectUnionMode union_mode_value) { union_mode = union_mode_value; } + /// Get union modes const SelectUnionModes & getUnionModes() const { return union_modes; } + /// Set union modes value void setUnionModes(const SelectUnionModes & union_modes_value) { union_modes = union_modes_value; union_modes_set = SelectUnionModesSet(union_modes.begin(), union_modes.end()); } - const QueryTreeNodePtr & getQueriesNode() const - { - return children[queries_child_index]; - } - - QueryTreeNodePtr & getQueriesNode() - { - return children[queries_child_index]; - } - + /// Get union node queries const ListNode & getQueries() const { return children[queries_child_index]->as(); } + /// Get union node queries ListNode & getQueries() { return children[queries_child_index]->as(); } + /// Get union node queries node + const QueryTreeNodePtr & getQueriesNode() const + { + return children[queries_child_index]; + } + + /// Get union node queries node + QueryTreeNodePtr & getQueriesNode() + { + return children[queries_child_index]; + } + /// Return true if union node has table expression modifiers, false otherwise bool hasTableExpressionModifiers() const { @@ -117,8 +140,7 @@ public: table_expression_modifiers = std::move(table_expression_modifiers_value); } - - /// Compute union projection + /// Compute union node projection columns NamesAndTypes computeProjectionColumns() const; QueryTreeNodeType getNodeType() const override From 144b33c75dec0385c5ae14c4965a9ef81a2ed276 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 18 Oct 2022 12:02:28 +0200 Subject: [PATCH 143/188] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 49 +++++++++---------- .../0_stateless/02372_analyzer_join.sql.j2 | 2 + 2 files changed, 26 insertions(+), 25 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 2ad9b84ef7c..afe89ce2696 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -18,6 +18,30 @@ #include #include +#include +#include +#include +#include + +#include + +#include + +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include + #include #include #include @@ -41,31 +65,6 @@ #include #include -#include - -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -#include -#include - -#include - -#include - - namespace DB { diff --git a/tests/queries/0_stateless/02372_analyzer_join.sql.j2 b/tests/queries/0_stateless/02372_analyzer_join.sql.j2 index 4259bad04ba..970c0d3da17 100644 --- a/tests/queries/0_stateless/02372_analyzer_join.sql.j2 +++ b/tests/queries/0_stateless/02372_analyzer_join.sql.j2 @@ -1,3 +1,5 @@ +-- Tags: long + SET use_analyzer = 1; DROP TABLE IF EXISTS test_table_join_1; From 1410befbe1d827a448387e23107e9cab5c597331 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 18 Oct 2022 13:14:56 +0200 Subject: [PATCH 144/188] Added AggregateFunctionsArithmeticOperationsPass --- ...egateFunctionsArithmericOperationsPass.cpp | 165 ++++++++++++++++++ ...gregateFunctionsArithmericOperationsPass.h | 24 +++ src/Analyzer/Passes/CountDistinctPass.h | 3 +- .../Passes/CustomizeFunctionsPass.cpp | 5 +- src/Analyzer/Passes/CustomizeFunctionsPass.h | 3 +- .../Passes/FunctionToSubcolumnsPass.h | 3 +- src/Analyzer/Passes/IfChainToMultiIfPass.h | 3 +- src/Analyzer/Passes/IfConstantConditionPass.h | 3 +- src/Analyzer/Passes/MultiIfToIfPass.h | 1 - .../Passes/NormalizeCountVariantsPass.h | 3 +- .../OrderByLimitByDuplicateEliminationPass.h | 3 +- .../Passes/OrderByTupleEliminationPass.h | 1 - src/Analyzer/Passes/SumIfToCountIfPass.h | 3 +- .../UniqInjectiveFunctionsEliminationPass.h | 7 +- src/Analyzer/QueryTreePassManager.cpp | 5 +- 15 files changed, 207 insertions(+), 25 deletions(-) create mode 100644 src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp create mode 100644 src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.h diff --git a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp new file mode 100644 index 00000000000..b90b624b147 --- /dev/null +++ b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp @@ -0,0 +1,165 @@ +#include + +#include +#include + +#include + +#include +#include + +namespace DB +{ + +namespace +{ + +Field zeroField(const Field & value) +{ + switch (value.getType()) + { + case Field::Types::UInt64: return static_cast(0); + case Field::Types::Int64: return static_cast(0); + case Field::Types::Float64: return static_cast(0); + case Field::Types::UInt128: return static_cast(0); + case Field::Types::Int128: return static_cast(0); + case Field::Types::UInt256: return static_cast(0); + case Field::Types::Int256: return static_cast(0); + default: + break; + } + + throw Exception("Unexpected literal type in function", ErrorCodes::BAD_TYPE_OF_FIELD); +} + +/** Rewrites: sum([multiply|divide]) -> [multiply|divide](sum) + * [min|max|avg]([multiply|divide|plus|minus]) -> [multiply|divide|plus|minus]([min|max|avg]) + * + * TODO: Support `groupBitAnd`, `groupBitOr`, `groupBitXor` functions. + * TODO: Support rewrite `f((2 * n) * n)` into '2 * f(n * n)'. + */ +class AggregateFunctionsArithmericOperationsVisitor : public InDepthQueryTreeVisitor +{ +public: + /// Traverse tree bottom to top + bool shouldTraverseTopToBottom() const + { + return false; + } + + void visitImpl(QueryTreeNodePtr & node) + { + auto * aggregate_function_node = node->as(); + if (!aggregate_function_node || !aggregate_function_node->isAggregateFunction()) + return; + + static std::unordered_map> supported_functions + = {{"sum", {"multiply", "divide"}}, + {"min", {"multiply", "divide", "plus", "minus"}}, + {"max", {"multiply", "divide", "plus", "minus"}}, + {"avg", {"multiply", "divide", "plus", "minus"}}}; + + auto & aggregate_function_arguments_nodes = aggregate_function_node->getArguments().getNodes(); + if (aggregate_function_arguments_nodes.size() != 1) + return; + + auto * inner_function_node = aggregate_function_arguments_nodes[0]->as(); + if (!inner_function_node) + return; + + auto & inner_function_arguments_nodes = inner_function_node->getArguments().getNodes(); + if (inner_function_arguments_nodes.size() != 2) + return; + + /// Aggregate functions[sum|min|max|avg] is case-insensitive, so we use lower cases name + auto lower_function_name = Poco::toLower(aggregate_function_node->getFunctionName()); + + auto supported_function_it = supported_functions.find(lower_function_name); + if (supported_function_it == supported_functions.end()) + return; + + const auto & inner_function_name = inner_function_node->getFunctionName(); + + if (!supported_function_it->second.contains(inner_function_name)) + return; + + auto left_argument_constant_value = inner_function_arguments_nodes[0]->getConstantValueOrNull(); + auto right_argument_constant_value = inner_function_arguments_nodes[1]->getConstantValueOrNull(); + + /** If we extract negative constant, aggregate function name must be updated. + * + * Example: SELECT min(-1 * id); + * Result: SELECT -1 * max(id); + */ + std::string function_name_if_constant_is_negative; + if (inner_function_name == "multiply" || inner_function_name == "divide") + { + if (lower_function_name == "min") + function_name_if_constant_is_negative = "max"; + else if (lower_function_name == "max") + function_name_if_constant_is_negative = "min"; + } + + if (left_argument_constant_value && !right_argument_constant_value) + { + /// Do not rewrite `sum(1/n)` with `sum(1) * div(1/n)` because of lose accuracy + if (inner_function_name == "divide") + return; + + /// Rewrite `aggregate_function(inner_function(constant, argument))` into `inner_function(constant, aggregate_function(argument))` + const auto & left_argument_constant_value_literal = left_argument_constant_value->getValue(); + if (!function_name_if_constant_is_negative.empty() && + left_argument_constant_value_literal < zeroField(left_argument_constant_value_literal)) + { + resolveAggregateFunctionNode(*aggregate_function_node, function_name_if_constant_is_negative); + } + + auto inner_function = aggregate_function_arguments_nodes[0]; + auto inner_function_right_argument = std::move(inner_function_arguments_nodes[1]); + aggregate_function_arguments_nodes = {inner_function_right_argument}; + inner_function_arguments_nodes[1] = node; + node = std::move(inner_function); + } + else if (right_argument_constant_value) + { + /// Rewrite `aggregate_function(inner_function(argument, constant))` into `inner_function(aggregate_function(argument), constant)` + const auto & right_argument_constant_value_literal = right_argument_constant_value->getValue(); + if (!function_name_if_constant_is_negative.empty() && + right_argument_constant_value_literal < zeroField(right_argument_constant_value_literal)) + { + resolveAggregateFunctionNode(*aggregate_function_node, function_name_if_constant_is_negative); + } + + auto inner_function = aggregate_function_arguments_nodes[0]; + auto inner_function_left_argument = std::move(inner_function_arguments_nodes[0]); + aggregate_function_arguments_nodes = {inner_function_left_argument}; + inner_function_arguments_nodes[0] = node; + node = std::move(inner_function); + } + } + +private: + static inline void resolveAggregateFunctionNode(FunctionNode & function_node, const String & aggregate_function_name) + { + auto function_result_type = function_node.getResultType(); + auto function_aggregate_function = function_node.getAggregateFunction(); + + AggregateFunctionProperties properties; + auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, + function_aggregate_function->getArgumentTypes(), + function_aggregate_function->getParameters(), + properties); + + function_node.resolveAsAggregateFunction(std::move(aggregate_function), std::move(function_result_type)); + } +}; + +} + +void AggregateFunctionsArithmericOperationsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) +{ + AggregateFunctionsArithmericOperationsVisitor visitor; + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.h b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.h new file mode 100644 index 00000000000..a89d2f87ad9 --- /dev/null +++ b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.h @@ -0,0 +1,24 @@ +#pragma once + +#include + +namespace DB +{ + +/** Extract arithmeric operations from aggregate functions. + * + * Example: SELECT sum(a * 2); + * Result: SELECT sum(a) * 2; + */ +class AggregateFunctionsArithmericOperationsPass final : public IQueryTreePass +{ +public: + String getName() override { return "AggregateFunctionsArithmericOperations"; } + + String getDescription() override { return "Extract arithmeric operations from aggregate functions."; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} diff --git a/src/Analyzer/Passes/CountDistinctPass.h b/src/Analyzer/Passes/CountDistinctPass.h index 0de5b3a9f13..cac5033c98f 100644 --- a/src/Analyzer/Passes/CountDistinctPass.h +++ b/src/Analyzer/Passes/CountDistinctPass.h @@ -13,7 +13,7 @@ namespace DB class CountDistinctPass final : public IQueryTreePass { public: - String getName() override { return "CountDistinctPass"; } + String getName() override { return "CountDistinct"; } String getDescription() override { @@ -25,4 +25,3 @@ public: }; } - diff --git a/src/Analyzer/Passes/CustomizeFunctionsPass.cpp b/src/Analyzer/Passes/CustomizeFunctionsPass.cpp index 6990b7d9d20..3c207bc35a1 100644 --- a/src/Analyzer/Passes/CustomizeFunctionsPass.cpp +++ b/src/Analyzer/Passes/CustomizeFunctionsPass.cpp @@ -3,12 +3,13 @@ #include #include -#include -#include #include #include +#include +#include + namespace DB { diff --git a/src/Analyzer/Passes/CustomizeFunctionsPass.h b/src/Analyzer/Passes/CustomizeFunctionsPass.h index dd334db2b3f..7145099ca4c 100644 --- a/src/Analyzer/Passes/CustomizeFunctionsPass.h +++ b/src/Analyzer/Passes/CustomizeFunctionsPass.h @@ -14,7 +14,7 @@ namespace DB class CustomizeFunctionsPass final : public IQueryTreePass { public: - String getName() override { return "CustomizeFunctionsPass"; } + String getName() override { return "CustomizeFunctions"; } String getDescription() override { return "Customize implementation of aggregate functions, and in functions."; } @@ -23,4 +23,3 @@ public: }; } - diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.h b/src/Analyzer/Passes/FunctionToSubcolumnsPass.h index d5c1dc3805d..e31c39a8ff3 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.h +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.h @@ -20,7 +20,7 @@ namespace DB class FunctionToSubcolumnsPass final : public IQueryTreePass { public: - String getName() override { return "FunctionToSubcolumnsPass"; } + String getName() override { return "FunctionToSubcolumns"; } String getDescription() override { return "Rewrite function to subcolumns, for example tupleElement(column, subcolumn) into column.subcolumn"; } @@ -29,4 +29,3 @@ public: }; } - diff --git a/src/Analyzer/Passes/IfChainToMultiIfPass.h b/src/Analyzer/Passes/IfChainToMultiIfPass.h index dc8fa36b54b..43f3fb8831d 100644 --- a/src/Analyzer/Passes/IfChainToMultiIfPass.h +++ b/src/Analyzer/Passes/IfChainToMultiIfPass.h @@ -14,7 +14,7 @@ namespace DB class IfChainToMultiIfPass final : public IQueryTreePass { public: - String getName() override { return "IfChainToMultiIfPass"; } + String getName() override { return "IfChainToMultiIf"; } String getDescription() override { return "Optimize if chain to multiIf"; } @@ -23,4 +23,3 @@ public: }; } - diff --git a/src/Analyzer/Passes/IfConstantConditionPass.h b/src/Analyzer/Passes/IfConstantConditionPass.h index 89fcddefcfa..571a27346c8 100644 --- a/src/Analyzer/Passes/IfConstantConditionPass.h +++ b/src/Analyzer/Passes/IfConstantConditionPass.h @@ -16,7 +16,7 @@ namespace DB class IfConstantConditionPass final : public IQueryTreePass { public: - String getName() override { return "IfConstantConditionPass"; } + String getName() override { return "IfConstantCondition"; } String getDescription() override { return "Optimize if, multiIf for constant condition."; } @@ -25,4 +25,3 @@ public: }; } - diff --git a/src/Analyzer/Passes/MultiIfToIfPass.h b/src/Analyzer/Passes/MultiIfToIfPass.h index d8200a03ea6..6791e8ec463 100644 --- a/src/Analyzer/Passes/MultiIfToIfPass.h +++ b/src/Analyzer/Passes/MultiIfToIfPass.h @@ -21,4 +21,3 @@ public: }; } - diff --git a/src/Analyzer/Passes/NormalizeCountVariantsPass.h b/src/Analyzer/Passes/NormalizeCountVariantsPass.h index ac9f8ab9537..974d96abd74 100644 --- a/src/Analyzer/Passes/NormalizeCountVariantsPass.h +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.h @@ -16,7 +16,7 @@ namespace DB class NormalizeCountVariantsPass final : public IQueryTreePass { public: - String getName() override { return "NormalizeCountVariantsPass"; } + String getName() override { return "NormalizeCountVariants"; } String getDescription() override { return "Optimize count(literal), sum(1) into count()."; } @@ -25,4 +25,3 @@ public: }; } - diff --git a/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h b/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h index b50099dbc6a..f0043a0bc78 100644 --- a/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h +++ b/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h @@ -15,7 +15,7 @@ namespace DB class OrderByLimitByDuplicateEliminationPass final : public IQueryTreePass { public: - String getName() override { return "OrderByLimitByDuplicateEliminationPass"; } + String getName() override { return "OrderByLimitByDuplicateElimination"; } String getDescription() override { return "Remove duplicate columns from ORDER BY, LIMIT BY."; } @@ -24,4 +24,3 @@ public: }; } - diff --git a/src/Analyzer/Passes/OrderByTupleEliminationPass.h b/src/Analyzer/Passes/OrderByTupleEliminationPass.h index 0809953ea4a..a25d52b8737 100644 --- a/src/Analyzer/Passes/OrderByTupleEliminationPass.h +++ b/src/Analyzer/Passes/OrderByTupleEliminationPass.h @@ -21,4 +21,3 @@ public: }; } - diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.h b/src/Analyzer/Passes/SumIfToCountIfPass.h index 4f5f189d4a3..f3ba47f1c2c 100644 --- a/src/Analyzer/Passes/SumIfToCountIfPass.h +++ b/src/Analyzer/Passes/SumIfToCountIfPass.h @@ -19,7 +19,7 @@ namespace DB class SumIfToCountIfPass final : public IQueryTreePass { public: - String getName() override { return "SumIfToCountIfPass"; } + String getName() override { return "SumIfToCountIf"; } String getDescription() override { return "Rewrite sum(if) and sumIf into countIf"; } @@ -28,4 +28,3 @@ public: }; } - diff --git a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h index 47fd21b26ba..a0f07dfb7b5 100644 --- a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h +++ b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h @@ -7,19 +7,18 @@ namespace DB /** Remove injective functions from `uniq*` functions arguments. * - * Example: SELECT uniq(injectiveFunction(argument)) + * Example: SELECT uniq(injectiveFunction(argument)); * Result: SELECT uniq(argument); */ class UniqInjectiveFunctionsEliminationPass final : public IQueryTreePass { public: - String getName() override { return "UniqInjectiveFunctionsEliminationPass"; } + String getName() override { return "UniqInjectiveFunctionsElimination"; } - String getDescription() override { return "Remove injective functions from uniq functions arguments"; } + String getDescription() override { return "Remove injective functions from uniq functions arguments."; } void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; }; } - diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 539bfcd76f1..05abbbfea69 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -31,7 +32,6 @@ namespace ErrorCodes * TODO: Support _shard_num into shardNum() rewriting. * TODO: Support logical expressions optimizer. * TODO: Support fuse sum count optimize_fuse_sum_count_avg, optimize_syntax_fuse_functions. - * TODO: Support setting optimize_arithmetic_operations_in_aggregate_functions. * TODO: Support setting convert_query_to_cnf. * TODO: Support setting optimize_using_constraints. * TODO: Support setting optimize_substitute_columns. @@ -130,6 +130,9 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_shared()); + if (settings.optimize_arithmetic_operations_in_aggregate_functions) + manager.addPass(std::make_shared()); + if (settings.optimize_injective_functions_inside_uniq) manager.addPass(std::make_shared()); From 82fae0125cf8a9b5ea7d87b2915cd3bb471b2247 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 18 Oct 2022 13:35:04 +0200 Subject: [PATCH 145/188] Added FullSortingMerge JOIN support --- src/Planner/PlannerJoinTree.cpp | 71 ++++++++++++++++++++++++++++++++- 1 file changed, 69 insertions(+), 2 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 9b05f78388f..e5958284f2d 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -20,6 +20,8 @@ #include #include +#include +#include #include #include #include @@ -320,7 +322,9 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, join_cast_plan_output_nodes(right_plan, right_plan_column_name_to_cast_type); const auto & query_context = planner_context->getQueryContext(); - bool join_use_nulls = query_context->getSettingsRef().join_use_nulls; + const auto & settings = query_context->getSettingsRef(); + + bool join_use_nulls = settings.join_use_nulls; auto to_nullable_function = FunctionFactory::instance().get("toNullable", query_context); auto join_cast_plan_columns_to_nullable = [&](QueryPlan & plan_to_add_cast) @@ -356,12 +360,15 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, } } - auto table_join = std::make_shared(query_context->getSettings(), query_context->getTemporaryVolume()); + auto table_join = std::make_shared(settings, query_context->getTemporaryVolume()); table_join->getTableJoin() = join_node.toASTTableJoin()->as(); table_join->getTableJoin().kind = join_kind; if (join_kind == JoinKind::Comma) + { + join_kind = JoinKind::Cross; table_join->getTableJoin().kind = JoinKind::Cross; + } table_join->setIsJoinWithConstant(join_constant != std::nullopt); @@ -511,6 +518,66 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, } else { + auto add_sorting = [&] (QueryPlan & plan, const Names & key_names, JoinTableSide join_table_side) + { + SortDescription sort_description; + sort_description.reserve(key_names.size()); + for (const auto & key_name : key_names) + sort_description.emplace_back(key_name); + + auto sorting_step = std::make_unique( + plan.getCurrentDataStream(), + std::move(sort_description), + settings.max_block_size, + 0 /*limit*/, + SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), + settings.max_bytes_before_remerge_sort, + settings.remerge_sort_lowered_memory_bytes_ratio, + settings.max_bytes_before_external_sort, + query_context->getTempDataOnDisk(), + settings.min_free_disk_space_for_temporary_data, + settings.optimize_sorting_by_input_stream_properties); + sorting_step->setStepDescription(fmt::format("Sort {} before JOIN", join_table_side)); + plan.addStep(std::move(sorting_step)); + }; + + auto crosswise_connection = CreateSetAndFilterOnTheFlyStep::createCrossConnection(); + auto add_create_set = [&settings, crosswise_connection](QueryPlan & plan, const Names & key_names, JoinTableSide join_table_side) + { + auto creating_set_step = std::make_unique( + plan.getCurrentDataStream(), + key_names, + settings.max_rows_in_set_to_optimize_join, + crosswise_connection, + join_table_side); + creating_set_step->setStepDescription(fmt::format("Create set and filter {} joined stream", join_table_side)); + + auto * step_raw_ptr = creating_set_step.get(); + plan.addStep(std::move(creating_set_step)); + return step_raw_ptr; + }; + + if (join_algorithm->pipelineType() == JoinPipelineType::YShaped) + { + const auto & join_clause = table_join->getOnlyClause(); + + bool kind_allows_filtering = isInner(join_kind) || isLeft(join_kind) || isRight(join_kind); + if (settings.max_rows_in_set_to_optimize_join > 0 && kind_allows_filtering) + { + auto * left_set = add_create_set(left_plan, join_clause.key_names_left, JoinTableSide::Left); + auto * right_set = add_create_set(right_plan, join_clause.key_names_right, JoinTableSide::Right); + + if (isInnerOrLeft(join_kind)) + right_set->setFiltering(left_set->getSet()); + + if (isInnerOrRight(join_kind)) + left_set->setFiltering(right_set->getSet()); + } + + add_sorting(left_plan, join_clause.key_names_left, JoinTableSide::Left); + add_sorting(right_plan, join_clause.key_names_right, JoinTableSide::Right); + } + size_t max_block_size = query_context->getSettingsRef().max_block_size; size_t max_streams = query_context->getSettingsRef().max_threads; From dbc0870eb5739577c71fcc30cb4d27dd036f6abc Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 18 Oct 2022 14:52:35 +0200 Subject: [PATCH 146/188] Fixed tests --- .../Passes/AggregateFunctionsArithmericOperationsPass.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp index b90b624b147..3be4c8d0bfa 100644 --- a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp +++ b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp @@ -11,6 +11,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_TYPE_OF_FIELD; +} + namespace { @@ -29,7 +34,7 @@ Field zeroField(const Field & value) break; } - throw Exception("Unexpected literal type in function", ErrorCodes::BAD_TYPE_OF_FIELD); + throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Unexpected literal type in function"); } /** Rewrites: sum([multiply|divide]) -> [multiply|divide](sum) From 90048ce75d6abc52b5fb3f7efa99fb71133d6d4a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 19 Oct 2022 12:25:27 +0200 Subject: [PATCH 147/188] Fixed code review issues --- src/Analyzer/AggregationUtils.cpp | 2 +- src/Analyzer/ArrayJoinNode.h | 2 + src/Analyzer/ColumnNode.h | 17 ++++--- src/Analyzer/ColumnTransformers.cpp | 2 + src/Analyzer/ColumnTransformers.h | 29 ++++++----- src/Analyzer/FunctionNode.cpp | 18 +++---- src/Analyzer/FunctionNode.h | 13 ++--- src/Analyzer/IQueryTreeNode.cpp | 9 ++-- src/Analyzer/IQueryTreeNode.h | 6 +-- src/Analyzer/IQueryTreePass.h | 4 +- src/Analyzer/IdentifierNode.cpp | 1 - src/Analyzer/InDepthQueryTreeVisitor.h | 2 +- src/Analyzer/InterpolateNode.cpp | 6 +-- src/Analyzer/JoinNode.cpp | 67 +++++++------------------ src/Analyzer/JoinNode.h | 33 ++++++------ src/Analyzer/LambdaNode.h | 18 +++---- src/Analyzer/MatcherNode.cpp | 49 ++++++++++++++---- src/Analyzer/MatcherNode.h | 28 ++++------- src/Analyzer/QueryNode.cpp | 14 +++++- src/Analyzer/QueryNode.h | 30 +++++------ src/Analyzer/QueryTreeBuilder.cpp | 33 ++++++------ src/Analyzer/QueryTreeBuilder.h | 2 +- src/Analyzer/QueryTreePassManager.cpp | 26 +++++----- src/Analyzer/QueryTreePassManager.h | 4 +- src/Analyzer/SetUtils.cpp | 8 +-- src/Analyzer/SortNode.cpp | 8 ++- src/Analyzer/SortNode.h | 11 ++-- src/Analyzer/TableExpressionModifiers.h | 9 +++- src/Analyzer/TableFunctionNode.h | 12 ++--- src/Analyzer/TableNode.h | 2 +- src/Analyzer/UnionNode.cpp | 46 +++++++++-------- src/Analyzer/UnionNode.h | 8 ++- src/Analyzer/Utils.cpp | 10 ++-- src/Analyzer/Utils.h | 3 +- src/Analyzer/WindowFunctionsUtils.cpp | 2 +- src/Analyzer/WindowNode.cpp | 4 ++ src/Analyzer/WindowNode.h | 22 ++++---- src/Parsers/ASTColumnsMatcher.cpp | 7 ++- 38 files changed, 303 insertions(+), 264 deletions(-) diff --git a/src/Analyzer/AggregationUtils.cpp b/src/Analyzer/AggregationUtils.cpp index 47ec15c647d..a73df87f9c2 100644 --- a/src/Analyzer/AggregationUtils.cpp +++ b/src/Analyzer/AggregationUtils.cpp @@ -34,7 +34,7 @@ public: if (!assert_no_aggregates_place_message.empty()) throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, "Aggregate function {} is found {} in query", - function_node->getName(), + function_node->formatASTForErrorMessage(), assert_no_aggregates_place_message); if (aggregate_function_nodes) diff --git a/src/Analyzer/ArrayJoinNode.h b/src/Analyzer/ArrayJoinNode.h index 7ca61b444e2..50d53df465a 100644 --- a/src/Analyzer/ArrayJoinNode.h +++ b/src/Analyzer/ArrayJoinNode.h @@ -73,11 +73,13 @@ public: return children[join_expressions_child_index]; } + /// Get join expressions node QueryTreeNodePtr & getJoinExpressionsNode() { return children[join_expressions_child_index]; } + /// Returns true if array join is left, false otherwise bool isLeft() const { return is_left; diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index 212285589d0..fbd788ae6fd 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -13,9 +13,8 @@ namespace ErrorCodes } /** Column node represents column in query tree. - * Column must have some column source. - * Column can be table expression, lambda, subquery. - * Column source must be valid during column node lifetime. + * Column node can have weak pointer to its column source. + * Column source can be table expression, lambda, subquery. * * For table ALIAS columns. Column node must contain expression. * For ARRAY JOIN join expression column. Column node must contain expression. @@ -28,8 +27,6 @@ namespace ErrorCodes * * Column node is initialized with column name, type and column source weak pointer. * In case of ALIAS column node is initialized with column name, type, alias expression and column source weak pointer. - * - * Additional care must be taken during clone to repoint column source to another node if its necessary see IQueryTreeNode.h `clone` method. */ class ColumnNode; using ColumnNodePtr = std::shared_ptr; @@ -37,10 +34,10 @@ using ColumnNodePtr = std::shared_ptr; class ColumnNode final : public IQueryTreeNode { public: - /// Construct expression column node with column name, type, column expression and column source weak pointer. + /// Construct column node with column name, type, column expression and column source weak pointer ColumnNode(NameAndTypePair column_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_); - /// Construct column node with column name, type and column source weak pointer. + /// Construct column node with column name, type and column source weak pointer ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_); /// Get column @@ -67,21 +64,25 @@ public: column.type = std::move(column_type); } + /// Returns true if column node has expression, false otherwise bool hasExpression() const { return children[expression_child_index] != nullptr; } + /// Get column node expression node const QueryTreeNodePtr & getExpression() const { return children[expression_child_index]; } + /// Get column node expression node QueryTreeNodePtr & getExpression() { return children[expression_child_index]; } + /// Get column node expression node, if there are no expression node exception is thrown QueryTreeNodePtr & getExpressionOrThrow() { if (!children[expression_child_index]) @@ -90,6 +91,7 @@ public: return children[expression_child_index]; } + /// Set column node expression node void setExpression(QueryTreeNodePtr expression_value) { children[expression_child_index] = std::move(expression_value); @@ -123,7 +125,6 @@ public: void dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t indent) const override; protected: - bool isEqualImpl(const IQueryTreeNode & rhs) const override; void updateTreeHashImpl(HashState & hash_state) const override; diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp index ac65094ea05..e8c9ccb1a6d 100644 --- a/src/Analyzer/ColumnTransformers.cpp +++ b/src/Analyzer/ColumnTransformers.cpp @@ -203,6 +203,8 @@ void ExceptColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & hash_state.update(static_cast(getTransformerType())); hash_state.update(static_cast(getExceptTransformerType())); + hash_state.update(except_column_names.size()); + for (const auto & column_name : except_column_names) { hash_state.update(column_name.size()); diff --git a/src/Analyzer/ColumnTransformers.h b/src/Analyzer/ColumnTransformers.h index fa45f2deee9..e96e606d923 100644 --- a/src/Analyzer/ColumnTransformers.h +++ b/src/Analyzer/ColumnTransformers.h @@ -12,11 +12,11 @@ namespace DB /** Transformers are query tree nodes that handle additional logic that you can apply after MatcherQueryTreeNode is resolved. * Check MatcherQueryTreeNode.h before reading this documentation. * - * They main purpose it to apply some logic for expressions after matcher is resolved. + * They main purpose is to apply some logic for expressions after matcher is resolved. * There are 3 types of transformers: * * 1. APPLY transformer: - * APPLY transformer transform expression using lambda or function into another expression. + * APPLY transformer transform matched expression using lambda or function into another expression. * It has 2 syntax variants: * 1. lambda variant: SELECT matcher APPLY (x -> expr(x)). * 2. function variant: SELECT matcher APPLY function_name(optional_parameters). @@ -111,7 +111,7 @@ class ApplyColumnTransformerNode final : public IColumnTransformerNode { public: /** Initialize apply column transformer with expression node. - * Expression node must be lambda or function otherwise exception is throwned. + * Expression node must be lambda or function otherwise exception is thrown. */ explicit ApplyColumnTransformerNode(QueryTreeNodePtr expression_node_); @@ -162,13 +162,13 @@ const char * toString(ExceptColumnTransformerType type); class ExceptColumnTransformerNode; using ExceptColumnTransformerNodePtr = std::shared_ptr; -/** Except column transformer - * Strict column transformer must use all column names during matched nodes transformation. +/** Except column transformer. + * Strict EXCEPT column transformer must use all column names during matched nodes transformation. * * Example: * CREATE TABLE test_table (id UInt64, value String) ENGINE=TinyLog; * SELECT * EXCEPT STRICT (id, value1) FROM test_table; - * Such query will throw exception because column name with value1 was not matched by strict EXCEPT transformer. + * Such query will throw exception because column with name `value1` was not matched by strict EXCEPT transformer. * * Strict is valid only for EXCEPT COLUMN_LIST transformer. */ @@ -187,7 +187,7 @@ public: return except_transformer_type; } - /** Get is except transformer strict. + /** Returns true if except column transformer is strict, false otherwise. * Valid only for EXCEPT COLUMN_LIST transformer. */ bool isStrict() const @@ -234,13 +234,13 @@ private: class ReplaceColumnTransformerNode; using ReplaceColumnTransformerNodePtr = std::shared_ptr; -/** Replace column transformer +/** Replace column transformer. * Strict replace column transformer must use all replacements during matched nodes transformation. * * Example: * CREATE TABLE test_table (id UInt64, value String) ENGINE=TinyLog; * SELECT * REPLACE STRICT (1 AS id, 2 AS value_1) FROM test_table; - * Such query will throw exception because column name with value1 was not matched by strict REPLACE transformer. + * Such query will throw exception because column with name `value1` was not matched by strict REPLACE transformer. */ class ReplaceColumnTransformerNode final : public IColumnTransformerNode { @@ -261,7 +261,7 @@ public: } /// Get replacements - ListNode & getReplacements() const + const ListNode & getReplacements() const { return children[replacements_child_index]->as(); } @@ -278,13 +278,13 @@ public: return replacements_names; } - /// Is replace column transformer strict + /// Returns true if replace column transformer is strict, false otherwise bool isStrict() const { return is_strict; } - /** Returns replacement expression if for expression name replacements exists, nullptr otherwise. + /** Returns replacement expression if replacement is registered for expression name, null otherwise. * Returned replacement expression must be cloned by caller. */ QueryTreeNodePtr findReplacementExpression(const std::string & expression_name); @@ -301,6 +301,11 @@ protected: ASTPtr toASTImpl() const override; private: + ListNode & getReplacements() + { + return children[replacements_child_index]->as(); + } + Names replacements_names; bool is_strict = false; diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index c5bb3d5c56d..7468141b3d5 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -189,15 +189,6 @@ ASTPtr FunctionNode::toASTImpl() const function_ast->name = function_name; function_ast->is_window_function = isWindowFunction(); - auto window_node = getWindowNode(); - if (window_node) - { - if (auto * identifier_node = window_node->as()) - function_ast->window_name = identifier_node->getIdentifier().getFullName(); - else - function_ast->window_definition = window_node->toAST(); - } - const auto & parameters = getParameters(); if (!parameters.getNodes().empty()) { @@ -209,6 +200,15 @@ ASTPtr FunctionNode::toASTImpl() const function_ast->children.push_back(arguments.toAST()); function_ast->arguments = function_ast->children.back(); + auto window_node = getWindowNode(); + if (window_node) + { + if (auto * identifier_node = window_node->as()) + function_ast->window_name = identifier_node->getIdentifier().getFullName(); + else + function_ast->window_definition = window_node->toAST(); + } + return function_ast; } diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index 40ab9a77d4d..18b4c6d445c 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -25,7 +25,7 @@ using AggregateFunctionPtr = std::shared_ptr; * 2. Non aggregate function. Example: plus(x, x). * 3. Window function. Example: sum(x) OVER (PARTITION BY expr ORDER BY expr). * - * Initially function node is initialize with function name. + * Initially function node is initialized with function name. * For window function client must initialize function window node. * * During query analysis pass function must be resolved using `resolveAsFunction`, `resolveAsAggregateFunction`, `resolveAsWindowFunction` methods. @@ -42,7 +42,7 @@ public: */ explicit FunctionNode(String function_name_); - /// Get name + /// Get function name const String & getFunctionName() const { return function_name; @@ -96,7 +96,7 @@ public: return children[arguments_child_index]; } - /// Has window + /// Returns true if function node has window, false otherwise bool hasWindow() const { return children[window_child_index] != nullptr; @@ -104,8 +104,9 @@ public: /** Get window node. * Valid only for window function node. - * Can be identifier if window function is defined as expr OVER window_name. - * Or can be window node if window function is defined as expr OVER (window_name ...). + * Result window node can be identifier node or window node. + * 1. It can be identifier node if window function is defined as expr OVER window_name. + * 2. It can be window node if window function is defined as expr OVER (window_name ...). */ const QueryTreeNodePtr & getWindowNode() const { @@ -164,7 +165,7 @@ public: /** Resolve function node as non aggregate function. * It is important that function name is updated with resolved function name. * Main motivation for this is query tree optimizations. - * Assume we have `multiIf` function with single argument, it can be converted to `if` function. + * Assume we have `multiIf` function with single condition, it can be converted to `if` function. * Function name must be updated accordingly. */ void resolveAsFunction(FunctionOverloadResolverPtr function_value, DataTypePtr result_type_value); diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 178abd4550b..dbc67749a8b 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -168,6 +168,8 @@ IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const nodes_to_process.push_back(node_to_process_child.get()); } + hash_state.update(node_to_process->weak_pointers.size()); + for (const auto & weak_pointer : node_to_process->weak_pointers) { auto strong_pointer = weak_pointer.lock(); @@ -282,9 +284,10 @@ String IQueryTreeNode::formatConvertedASTForErrorMessage() const String IQueryTreeNode::dumpTree() const { - WriteBufferFromOwnString buff; - dumpTree(buff); - return buff.str(); + WriteBufferFromOwnString buffer; + dumpTree(buffer); + + return buffer.str(); } size_t IQueryTreeNode::FormatState::getNodeId(const IQueryTreeNode * node) diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 63b00085092..2ba96d27575 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -53,7 +53,7 @@ const char * toString(QueryTreeNodeType type); /** Query tree is semantical representation of query. * Query tree node represent node in query tree. - * Query tree node is base class for all query tree nodes. + * IQueryTreeNode is base class for all query tree nodes. * * Important property of query tree is that each query tree node can contain weak pointers to other * query tree nodes. Keeping weak pointer to other query tree nodes can be useful for example for column @@ -167,7 +167,7 @@ public: alias = {}; } - /// Check if query tree node has original AST + /// Returns true if query tree node has original AST, false otherwise bool hasOriginalAST() const { return original_ast != nullptr; @@ -188,7 +188,7 @@ public: } /** If query tree has original AST format it for error message. - * Otherwise throw an exception. + * Otherwise exception is thrown. */ String formatOriginalASTForErrorMessage() const; diff --git a/src/Analyzer/IQueryTreePass.h b/src/Analyzer/IQueryTreePass.h index 9c565438dde..39b3d743ed3 100644 --- a/src/Analyzer/IQueryTreePass.h +++ b/src/Analyzer/IQueryTreePass.h @@ -8,11 +8,11 @@ namespace DB { -/** After query tree is build it can be later processed by QueryTreePassManager. +/** After query tree is build it can be later processed by query tree passes. * This is abstract base class for all query tree passes. * * Query tree pass can make query tree modifications, after each pass query tree must be valid. - * Query tree pass must perform be isolated and perform only necessary query tree modifications for doing its job. + * Query tree pass must be isolated and perform only necessary query tree modifications for doing its job. * Dependencies between passes must be avoided. */ class IQueryTreePass; diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp index 6df3ad0fd51..4efc7f515ea 100644 --- a/src/Analyzer/IdentifierNode.cpp +++ b/src/Analyzer/IdentifierNode.cpp @@ -48,7 +48,6 @@ bool IdentifierNode::isEqualImpl(const IQueryTreeNode & rhs) const else if (!table_expression_modifiers && rhs_typed.table_expression_modifiers) return false; - return identifier == rhs_typed.identifier; } diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index fd0d8e9d10f..96972024d87 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -9,7 +9,7 @@ namespace DB { /** Visitor that traverse query tree in depth. - * Derived class must implement `visitImpl` methods. + * Derived class must implement `visitImpl` method. * Additionally subclass can control if child need to be visited using `needChildVisit` method, by * default all node children are visited. * By default visitor traverse tree from top to bottom, if bottom to top traverse is required subclass diff --git a/src/Analyzer/InterpolateNode.cpp b/src/Analyzer/InterpolateNode.cpp index c6290aa6802..dcc14d6b6d5 100644 --- a/src/Analyzer/InterpolateNode.cpp +++ b/src/Analyzer/InterpolateNode.cpp @@ -28,7 +28,7 @@ String InterpolateNode::getName() const void InterpolateNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { - buffer << std::string(indent, ' ') << "INTERPOLATE_COLUMN id: " << format_state.getNodeId(this); + buffer << std::string(indent, ' ') << "INTERPOLATE id: " << format_state.getNodeId(this); buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION\n"; getExpression()->dumpTreeImpl(buffer, format_state, indent + 4); @@ -39,13 +39,13 @@ void InterpolateNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_st bool InterpolateNode::isEqualImpl(const IQueryTreeNode &) const { - /// No state in interpolate column node + /// No state in interpolate node return true; } void InterpolateNode::updateTreeHashImpl(HashState &) const { - /// No state in interpolate column node + /// No state in interpolate node } QueryTreeNodePtr InterpolateNode::cloneImpl() const diff --git a/src/Analyzer/JoinNode.cpp b/src/Analyzer/JoinNode.cpp index 315af809e7f..28a0c4ad7e0 100644 --- a/src/Analyzer/JoinNode.cpp +++ b/src/Analyzer/JoinNode.cpp @@ -31,21 +31,24 @@ JoinNode::JoinNode(QueryTreeNodePtr left_table_expression_, children[join_expression_child_index] = std::move(join_expression_); } -JoinNode::JoinNode(QueryTreeNodePtr left_table_expression_, - QueryTreeNodePtr right_table_expression_, - QueryTreeNodes using_identifiers, - JoinLocality locality_, - JoinStrictness strictness_, - JoinKind kind_) - : IQueryTreeNode(children_size) - , locality(locality_) - , strictness(strictness_) - , kind(kind_) +ASTPtr JoinNode::toASTTableJoin() const { - children.resize(children_size); - children[left_table_expression_child_index] = std::move(left_table_expression_); - children[right_table_expression_child_index] = std::move(right_table_expression_); - children[join_expression_child_index] = std::make_shared(std::move(using_identifiers)); + auto join_ast = std::make_shared(); + join_ast->locality = locality; + join_ast->strictness = strictness; + join_ast->kind = kind; + + if (children[join_expression_child_index]) + { + auto join_expression_ast = children[join_expression_child_index]->toAST(); + + if (children[join_expression_child_index]->getNodeType() == QueryTreeNodeType::LIST) + join_ast->using_expression_list = std::move(join_expression_ast); + else + join_ast->on_expression = std::move(join_expression_ast); + } + + return join_ast; } void JoinNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const @@ -86,26 +89,6 @@ void JoinNode::updateTreeHashImpl(HashState & state) const state.update(kind); } -ASTPtr JoinNode::toASTTableJoin() const -{ - auto join_ast = std::make_shared(); - join_ast->locality = locality; - join_ast->strictness = strictness; - join_ast->kind = kind; - - if (children[join_expression_child_index]) - { - auto join_expression_ast = children[join_expression_child_index]->toAST(); - - if (children[join_expression_child_index]->getNodeType() == QueryTreeNodeType::LIST) - join_ast->using_expression_list = std::move(join_expression_ast); - else - join_ast->on_expression = std::move(join_expression_ast); - } - - return join_ast; -} - QueryTreeNodePtr JoinNode::cloneImpl() const { return std::make_shared(getLeftTableExpression(), getRightTableExpression(), getJoinExpression(), locality, strictness, kind); @@ -119,21 +102,7 @@ ASTPtr JoinNode::toASTImpl() const size_t join_table_index = tables_in_select_query_ast->children.size(); - auto join_ast = std::make_shared(); - join_ast->locality = locality; - join_ast->strictness = strictness; - join_ast->kind = kind; - - if (children[join_expression_child_index]) - { - auto join_expression_ast = children[join_expression_child_index]->toAST(); - join_ast->children.push_back(std::move(join_expression_ast)); - - if (children[join_expression_child_index]->getNodeType() == QueryTreeNodeType::LIST) - join_ast->using_expression_list = join_ast->children.back(); - else - join_ast->on_expression = join_ast->children.back(); - } + auto join_ast = toASTTableJoin(); addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, children[right_table_expression_child_index]); diff --git a/src/Analyzer/JoinNode.h b/src/Analyzer/JoinNode.h index 36ba03f6f6c..15ba11a0122 100644 --- a/src/Analyzer/JoinNode.h +++ b/src/Analyzer/JoinNode.h @@ -24,7 +24,7 @@ namespace DB * Example: SELECT id FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 USING (id); * * For JOIN with ON, JOIN expression contains single expression. - * Example: SELECT id FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t1.id; + * Example: SELECT id FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id; */ class JoinNode; using JoinNodePtr = std::shared_ptr; @@ -37,7 +37,7 @@ public: * * test_table_1 - left table expression. * test_table_2 - right table expression. - * join_expression - join_expression; + * expression - join expression. */ JoinNode(QueryTreeNodePtr left_table_expression_, QueryTreeNodePtr right_table_expression_, @@ -46,19 +46,6 @@ public: JoinStrictness strictness_, JoinKind kind_); - /** Construct join node with left table expression, right table expression and using identifiers. - * Example: SELECT id FROM test_table_1 INNER JOIN test_table_2 USING (using_identifier, ...). - * test_table_1 - left table expression. - * test_table_2 - right table expression. - * (using_identifier, ...) - using identifiers. - */ - JoinNode(QueryTreeNodePtr left_table_expression_, - QueryTreeNodePtr right_table_expression_, - QueryTreeNodes using_identifiers, - JoinLocality locality_, - JoinStrictness strictness_, - JoinKind kind_); - /// Get left table expression const QueryTreeNodePtr & getLeftTableExpression() const { @@ -83,6 +70,12 @@ public: return children[right_table_expression_child_index]; } + /// Returns true if join has join expression, false otherwise + bool hasJoinExpression() const + { + return children[join_expression_child_index] != nullptr; + } + /// Get join expression const QueryTreeNodePtr & getJoinExpression() const { @@ -95,31 +88,37 @@ public: return children[join_expression_child_index]; } + /// Returns true if join has USING join expression, false otherwise bool isUsingJoinExpression() const { - return getJoinExpression() && getJoinExpression()->getNodeType() == QueryTreeNodeType::LIST; + return hasJoinExpression() && getJoinExpression()->getNodeType() == QueryTreeNodeType::LIST; } + /// Returns true if join has ON join expression, false otherwise bool isOnJoinExpression() const { - return getJoinExpression() && getJoinExpression()->getNodeType() != QueryTreeNodeType::LIST; + return hasJoinExpression() && getJoinExpression()->getNodeType() != QueryTreeNodeType::LIST; } + /// Get join locality JoinLocality getLocality() const { return locality; } + /// Get join strictness JoinStrictness getStrictness() const { return strictness; } + /// Get join kind JoinKind getKind() const { return kind; } + /// Convert join node to ASTTableJoin ASTPtr toASTTableJoin() const; QueryTreeNodeType getNodeType() const override diff --git a/src/Analyzer/LambdaNode.h b/src/Analyzer/LambdaNode.h index 3b41cdc9c78..6061e854ab0 100644 --- a/src/Analyzer/LambdaNode.h +++ b/src/Analyzer/LambdaNode.h @@ -12,13 +12,11 @@ namespace DB /** Lambda node represents lambda expression in query tree. * * Lambda consist of argument names and lambda expression body. - * Lambda expression body does not necessary use lambda arguments. Example: SELECT arrayMap(x -> 1, [1, 2, 3]) + * Lambda expression body does not necessary use lambda arguments. Example: SELECT arrayMap(x -> 1, [1, 2, 3]) * - * Initially lambda is initialized with argument names and expression query tree node. - * During query analysis if expression is not resolved lambda must be resolved. - * Lambda is in resolved state if lambda body expression is in resolved state. + * Initially lambda is initialized with argument names and lambda body expression. * - * It is important that lambda expression result type can depend on arguments types. + * Lambda expression result type can depend on arguments types. * Example: WITH (x -> x) as lambda SELECT lambda(1), lambda('string_value'). * * During query analysis pass lambdas must be resolved. @@ -27,8 +25,8 @@ namespace DB * If client modified lambda arguments array its size must be equal to initial lambda argument names array. * * Examples: - * WITH (x -> x + 1) as lambda SELECT lambda(1). - * SELECT arrayMap(x -> x + 1, [1,2,3]). + * WITH (x -> x + 1) as lambda SELECT lambda(1); + * SELECT arrayMap(x -> x + 1, [1,2,3]); */ class LambdaNode; using LambdaNodePtr = std::shared_ptr; @@ -36,7 +34,7 @@ using LambdaNodePtr = std::shared_ptr; class LambdaNode final : public IQueryTreeNode { public: - /// Initialize lambda with argument names and expression query tree node + /// Initialize lambda with argument names and lambda body expression explicit LambdaNode(Names argument_names_, QueryTreeNodePtr expression_); /// Get argument names @@ -63,9 +61,7 @@ public: return children[arguments_child_index]; } - /** Get arguments node. - * If arguments array is modified its result size must be equal to lambd argument names size. - */ + /// Get arguments node QueryTreeNodePtr & getArgumentsNode() { return children[arguments_child_index]; diff --git a/src/Analyzer/MatcherNode.cpp b/src/Analyzer/MatcherNode.cpp index 16633be13df..0bed9110d25 100644 --- a/src/Analyzer/MatcherNode.cpp +++ b/src/Analyzer/MatcherNode.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB { @@ -116,7 +117,7 @@ bool MatcherNode::isMatchingColumn(const std::string & column_name) if (columns_matcher) return RE2::PartialMatch(column_name, *columns_matcher); - return columns_identifiers_set.find(column_name) != columns_identifiers_set.end(); + return columns_identifiers_set.contains(column_name); } void MatcherNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const @@ -227,7 +228,7 @@ void MatcherNode::updateTreeHashImpl(HashState & hash_state) const { const auto & identifier_full_name = identifier.getFullName(); hash_state.update(identifier_full_name.size()); - hash_state.update(identifier_full_name.data(), identifier_full_name.size()); + hash_state.update(identifier_full_name); } if (columns_matcher) @@ -257,7 +258,6 @@ ASTPtr MatcherNode::toASTImpl() const if (matcher_type == MatcherNodeType::ASTERISK) { - /// For COLUMNS qualified identifier is not supported if (qualified_identifier.empty()) { result = std::make_shared(); @@ -265,6 +265,7 @@ ASTPtr MatcherNode::toASTImpl() const else { auto qualified_asterisk = std::make_shared(); + auto identifier_parts = qualified_identifier.getParts(); qualified_asterisk->children.push_back(std::make_shared(std::move(identifier_parts))); @@ -273,22 +274,50 @@ ASTPtr MatcherNode::toASTImpl() const } else if (columns_matcher) { - auto regexp_matcher = std::make_shared(); - regexp_matcher->setPattern(columns_matcher->pattern()); - result = regexp_matcher; + if (qualified_identifier.empty()) + { + auto regexp_matcher = std::make_shared(); + regexp_matcher->setPattern(columns_matcher->pattern()); + result = regexp_matcher; + } + else + { + auto regexp_matcher = std::make_shared(); + regexp_matcher->setPattern(columns_matcher->pattern()); + + auto identifier_parts = qualified_identifier.getParts(); + regexp_matcher->children.push_back(std::make_shared(std::move(identifier_parts))); + + result = regexp_matcher; + } } else { - auto columns_list_matcher = std::make_shared(); - columns_list_matcher->children.reserve(columns_identifiers.size()); + auto column_list = std::make_shared(); + column_list->children.reserve(columns_identifiers.size()); for (const auto & identifier : columns_identifiers) { auto identifier_parts = identifier.getParts(); - columns_list_matcher->children.push_back(std::make_shared(std::move(identifier_parts))); + column_list->children.push_back(std::make_shared(std::move(identifier_parts))); } - result = columns_list_matcher; + if (qualified_identifier.empty()) + { + auto columns_list_matcher = std::make_shared(); + columns_list_matcher->column_list = std::move(column_list); + result = columns_list_matcher; + } + else + { + auto columns_list_matcher = std::make_shared(); + columns_list_matcher->column_list = std::move(column_list); + + auto identifier_parts = qualified_identifier.getParts(); + columns_list_matcher->children.push_back(std::make_shared(std::move(identifier_parts))); + + result = columns_list_matcher; + } } for (const auto & child : children) diff --git a/src/Analyzer/MatcherNode.h b/src/Analyzer/MatcherNode.h index f989ac9bb7e..3635d038549 100644 --- a/src/Analyzer/MatcherNode.h +++ b/src/Analyzer/MatcherNode.h @@ -25,20 +25,13 @@ namespace DB * 2. identifier.COLUMNS('regexp') * 3. identifier.COLUMNS(column_name_1, ...) * - * The main difference between matcher and identifier is that matcher cannot have alias. - * This simplifies analysis for matchers. + * Matcher must be resolved during query analysis pass. * - * How to resolve matcher during query analysis pass: - * 1. If matcher is unqualified, we use tables of current scope and try to resolve matcher from it. - * 2. If matcher is qualified: - * First try to resolve identifier part as query expression. - * Try expressions from aliases, then from tables (can be changed using prefer_column_name_to_alias setting). - * If identifier is resolved as expression. If expression is compound apply matcher to it, otherwise throw exception. + * Matchers can be applied to compound expressions. * Example: SELECT compound_column AS a, a.* FROM test_table. * Example: SELECT compound_column.* FROM test_table. * - * If identifier is not resolved as expression try to resolve it as table. - * If identifier is resolved as table then apply matcher to it. + * Example: SELECT * FROM test_table; * Example: SELECT test_table.* FROM test_table. * Example: SELECT a.* FROM test_table AS a. * @@ -84,25 +77,25 @@ public: return matcher_type; } - /// Is this matcher represented by asterisk + /// Returns true if matcher is asterisk matcher, false otherwise bool isAsteriskMatcher() const { return matcher_type == MatcherNodeType::ASTERISK; } - /// Is this matcher represented by COLUMNS + /// Returns true if matcher is columns regexp or columns list matcher, false otherwise bool isColumnsMatcher() const { return matcher_type == MatcherNodeType::COLUMNS_REGEXP || matcher_type == MatcherNodeType::COLUMNS_LIST; } - /// Returns true if matcher qualified with identifier, false otherwise + /// Returns true if matcher is qualified, false otherwise bool isQualified() const { return !qualified_identifier.empty(); } - /// Returns true if matcher is not qualified with identifier, false otherwise + /// Returns true if matcher is not qualified, false otherwise bool isUnqualified() const { return qualified_identifier.empty(); @@ -120,20 +113,19 @@ public: return columns_matcher; } - /// Get columns matcher. Valid only if this matcher has type COLUMNS_LIST. + /// Get columns identifiers. Valid only if this matcher has type COLUMNS_LIST. const Identifiers & getColumnsIdentifiers() const { return columns_identifiers; } - /** Get column transformers - * Client can expect that node in this list is subclass of IColumnTransformerNode. - */ + /// Get column transformers const ListNode & getColumnTransformers() const { return children[column_transformers_child_index]->as(); } + /// Get column transformers const QueryTreeNodePtr & getColumnTransformersNode() const { return children[column_transformers_child_index]; diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 7cde154dc6f..378bd7259bf 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -299,6 +299,7 @@ bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name && + projection_columns == rhs_typed.projection_columns && is_distinct == rhs_typed.is_distinct && is_limit_with_ties == rhs_typed.is_limit_with_ties && is_group_by_with_totals == rhs_typed.is_group_by_with_totals && @@ -315,6 +316,17 @@ void QueryNode::updateTreeHashImpl(HashState & state) const state.update(cte_name.size()); state.update(cte_name); + state.update(projection_columns.size()); + for (const auto & projection_column : projection_columns) + { + state.update(projection_column.name.size()); + state.update(projection_column.name); + + auto projection_column_type_name = projection_column.type->getName(); + state.update(projection_column_type_name.size()); + state.update(projection_column_type_name); + } + state.update(is_distinct); state.update(is_limit_with_ties); state.update(is_group_by_with_totals); @@ -370,7 +382,7 @@ ASTPtr QueryNode::toASTImpl() const if (hasWith()) select_query->setExpression(ASTSelectQuery::Expression::WITH, getWith().toAST()); - select_query->setExpression(ASTSelectQuery::Expression::SELECT, children[projection_child_index]->toAST()); + select_query->setExpression(ASTSelectQuery::Expression::SELECT, getProjection().toAST()); ASTPtr tables_in_select_query_ast = std::make_shared(); addTableExpressionOrJoinIntoTablesInSelectQuery(tables_in_select_query_ast, getJoinTree()); diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 6ab9e5a72bd..05d393b4212 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -183,7 +183,7 @@ public: } /// Get table expression modifiers - std::optional getTableExpressionModifiers() const + const std::optional & getTableExpressionModifiers() const { return table_expression_modifiers; } @@ -194,7 +194,7 @@ public: table_expression_modifiers = std::move(table_expression_modifiers_value); } - /// Returns true if query node WITH section is not empty + /// Returns true if query node WITH section is not empty, false otherwise bool hasWith() const { return !getWith().getNodes().empty(); @@ -260,7 +260,7 @@ public: return children[join_tree_child_index]; } - /// Returns true if query node PREWHERE section is not empty + /// Returns true if query node PREWHERE section is not empty, false otherwise bool hasPrewhere() const { return children[prewhere_child_index] != nullptr; @@ -278,7 +278,7 @@ public: return children[prewhere_child_index]; } - /// Returns true if query node WHERE section is not empty + /// Returns true if query node WHERE section is not empty, false otherwise bool hasWhere() const { return children[where_child_index] != nullptr; @@ -296,7 +296,7 @@ public: return children[where_child_index]; } - /// Returns true if query node GROUP BY section is not empty + /// Returns true if query node GROUP BY section is not empty, false otherwise bool hasGroupBy() const { return !getGroupBy().getNodes().empty(); @@ -305,7 +305,7 @@ public: /// Get GROUP BY section const ListNode & getGroupBy() const { - return children[group_by_child_index]->as(); + return children[group_by_child_index]->as(); } /// Get GROUP BY section @@ -326,7 +326,7 @@ public: return children[group_by_child_index]; } - /// Returns true if query node HAVING section is not empty + /// Returns true if query node HAVING section is not empty, false otherwise bool hasHaving() const { return getHaving() != nullptr; @@ -344,7 +344,7 @@ public: return children[having_child_index]; } - /// Returns true if query node WINDOW section is not empty + /// Returns true if query node WINDOW section is not empty, false otherwise bool hasWindow() const { return !getWindow().getNodes().empty(); @@ -374,7 +374,7 @@ public: return children[window_child_index]; } - /// Returns true if query node ORDER BY section is not empty + /// Returns true if query node ORDER BY section is not empty, false otherwise bool hasOrderBy() const { return !getOrderBy().getNodes().empty(); @@ -404,7 +404,7 @@ public: return children[order_by_child_index]; } - /// Returns true if query node INTERPOLATE section is not empty + /// Returns true if query node INTERPOLATE section is not empty, false otherwise bool hasInterpolate() const { return getInterpolate() != nullptr; @@ -422,7 +422,7 @@ public: return children[interpolate_child_index]; } - /// Returns true if query node LIMIT BY LIMIT section is not empty + /// Returns true if query node LIMIT BY LIMIT section is not empty, false otherwise bool hasLimitByLimit() const { return children[limit_by_limit_child_index] != nullptr; @@ -440,7 +440,7 @@ public: return children[limit_by_limit_child_index]; } - /// Returns true if query node LIMIT BY OFFSET section is not empty + /// Returns true if query node LIMIT BY OFFSET section is not empty, false otherwise bool hasLimitByOffset() const { return children[limit_by_offset_child_index] != nullptr; @@ -458,7 +458,7 @@ public: return children[limit_by_offset_child_index]; } - /// Returns true if query node LIMIT BY section is not empty + /// Returns true if query node LIMIT BY section is not empty, false otherwise bool hasLimitBy() const { return !getLimitBy().getNodes().empty(); @@ -488,7 +488,7 @@ public: return children[limit_by_child_index]; } - /// Returns true if query node LIMIT section is not empty + /// Returns true if query node LIMIT section is not empty, false otherwise bool hasLimit() const { return children[limit_child_index] != nullptr; @@ -506,7 +506,7 @@ public: return children[limit_child_index]; } - /// Returns true if query node OFFSET section is not empty + /// Returns true if query node OFFSET section is not empty, false otherwise bool hasOffset() const { return children[offset_child_index] != nullptr; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 09baa1e98b8..890aa2b01a2 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -87,7 +87,7 @@ private: QueryTreeNodePtr buildSortList(const ASTPtr & order_by_expression_list) const; - QueryTreeNodePtr buildInterpolateColumnList(const ASTPtr & interpolate_expression_list) const; + QueryTreeNodePtr buildInterpolateList(const ASTPtr & interpolate_expression_list) const; QueryTreeNodePtr buildWindowList(const ASTPtr & window_definition_list) const; @@ -147,6 +147,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectWithUnionExpression(const ASTPtr & auto union_node = std::make_shared(); union_node->setIsSubquery(is_subquery); + union_node->setIsCTE(!cte_name.empty()); union_node->setCTEName(cte_name); union_node->setUnionMode(select_with_union_query_typed.union_mode); union_node->setUnionModes(select_with_union_query_typed.list_of_modes); @@ -174,6 +175,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectIntersectExceptQuery(const ASTPtr auto union_node = std::make_shared(); union_node->setIsSubquery(is_subquery); + union_node->setIsCTE(!cte_name.empty()); union_node->setCTEName(cte_name); if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::INTERSECT_ALL) @@ -278,7 +280,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q auto interpolate_list = select_query_typed.interpolate(); if (interpolate_list) - current_query_tree->getInterpolate() = buildInterpolateColumnList(interpolate_list); + current_query_tree->getInterpolate() = buildInterpolateList(interpolate_list); auto select_limit_by_limit = select_query_typed.limitByLength(); if (select_limit_by_limit) @@ -321,7 +323,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_express std::shared_ptr collator; if (order_by_element.collation) - collator = std::make_shared(order_by_element.collation->as().value.get()); + collator = std::make_shared(order_by_element.collation->as().value.get()); const auto & sort_expression_ast = order_by_element.children.at(0); auto sort_expression = buildExpression(sort_expression_ast); @@ -344,7 +346,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_express return list_node; } -QueryTreeNodePtr QueryTreeBuilder::buildInterpolateColumnList(const ASTPtr & interpolate_expression_list) const +QueryTreeNodePtr QueryTreeBuilder::buildInterpolateList(const ASTPtr & interpolate_expression_list) const { auto list_node = std::make_shared(); @@ -406,24 +408,19 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co if (const auto * ast_identifier = expression->as()) { - /// TODO: Identifier as query parameter auto identifier = Identifier(ast_identifier->name_parts); result = std::make_shared(std::move(identifier)); } else if (const auto * asterisk = expression->as()) { auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/); - result = std::make_shared(column_transformers); + result = std::make_shared(std::move(column_transformers)); } else if (const auto * qualified_asterisk = expression->as()) { - /// TODO: Identifier with UUID - /// TODO: Currently during query analysis stage we support qualified matchers with any identifier length - /// but ASTTableIdentifier can contain only 2 parts. - auto & qualified_identifier = qualified_asterisk->children.at(0)->as(); auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/); - result = std::make_shared(Identifier(qualified_identifier.name_parts), column_transformers); + result = std::make_shared(Identifier(qualified_identifier.name_parts), std::move(column_transformers)); } else if (const auto * ast_literal = expression->as()) { @@ -442,7 +439,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co if (lambda_arguments_tuple.arguments) { - const auto & lambda_arguments_list = lambda_arguments_tuple.arguments->as()->children; + const auto & lambda_arguments_list = lambda_arguments_tuple.arguments->as().children; for (const auto & lambda_argument : lambda_arguments_list) { const auto * lambda_argument_identifier = lambda_argument->as(); @@ -454,7 +451,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co if (lambda_argument_identifier->name_parts.size() > 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Lambda {} argument identifier must contain only argument name. Actual {}", + "Lambda {} argument identifier must contain single part. Actual {}", function->formatForErrorMessage(), lambda_argument_identifier->full_name); @@ -501,7 +498,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co function_node->getWindowNode() = std::make_shared(Identifier(function->window_name)); } - result = function_node; + result = std::move(function_node); } } else if (const auto * subquery = expression->as()) @@ -509,14 +506,14 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co auto subquery_query = subquery->children[0]; auto query_node = buildSelectWithUnionExpression(subquery_query, true /*is_subquery*/, {} /*cte_name*/); - result = query_node; + result = std::move(query_node); } else if (const auto * with_element = expression->as()) { auto with_element_subquery = with_element->subquery->as().children.at(0); auto query_node = buildSelectWithUnionExpression(with_element_subquery, true /*is_subquery*/, with_element->name /*cte_name*/); - result = query_node; + result = std::move(query_node); } else if (const auto * columns_regexp_matcher = expression->as()) { @@ -557,7 +554,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co } auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/); - result = std::make_shared(Identifier(qualified_identifier.name_parts), column_list_identifiers, std::move(column_transformers)); + result = std::make_shared(Identifier(qualified_identifier.name_parts), std::move(column_list_identifiers), std::move(column_transformers)); } else { @@ -699,7 +696,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select if (table_function_expression.arguments) { - const auto & function_arguments_list = table_function_expression.arguments->as()->children; + const auto & function_arguments_list = table_function_expression.arguments->as().children; for (const auto & argument : function_arguments_list) { if (argument->as() || argument->as() || argument->as()) diff --git a/src/Analyzer/QueryTreeBuilder.h b/src/Analyzer/QueryTreeBuilder.h index 3c02527436b..de0f6270230 100644 --- a/src/Analyzer/QueryTreeBuilder.h +++ b/src/Analyzer/QueryTreeBuilder.h @@ -10,7 +10,7 @@ namespace DB { /** Build query tree from AST. - * AST that represent query ASTSelectWithUnionQuery, ASTSelectQuery. + * AST that represent query ASTSelectWithUnionQuery, ASTSelectIntersectExceptQuery, ASTSelectQuery. * AST that represent a list of expressions ASTExpressionList. * AST that represent expression ASTIdentifier, ASTAsterisk, ASTLiteral, ASTFunction. */ diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 05abbbfea69..853b4a23f38 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -27,7 +27,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -/** ClickHouse query tree pass manager +/** ClickHouse query tree pass manager. * * TODO: Support _shard_num into shardNum() rewriting. * TODO: Support logical expressions optimizer. @@ -58,20 +58,20 @@ void QueryTreePassManager::addPass(QueryTreePassPtr pass) void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node) { auto current_context = getContext(); - size_t optimizations_size = passes.size(); + size_t passes_size = passes.size(); - for (size_t i = 0; i < optimizations_size; ++i) + for (size_t i = 0; i < passes_size; ++i) passes[i]->run(query_tree_node, current_context); } void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node, size_t up_to_pass_index) { - size_t optimizations_size = passes.size(); - if (up_to_pass_index > optimizations_size) + size_t passes_size = passes.size(); + if (up_to_pass_index > passes_size) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Requested to run optimizations up to {} pass. There are only {} pass", + "Requested to run passes up to {} pass. There are only {} passes", up_to_pass_index, - optimizations_size); + passes_size); auto current_context = getContext(); for (size_t i = 0; i < up_to_pass_index; ++i) @@ -86,25 +86,25 @@ void QueryTreePassManager::dump(WriteBuffer & buffer) { auto & pass = passes[i]; buffer << "Pass " << (i + 1) << ' ' << pass->getName() << " - " << pass->getDescription(); - if (i < passes_size) + if (i + 1 != passes_size) buffer << '\n'; } } void QueryTreePassManager::dump(WriteBuffer & buffer, size_t up_to_pass_index) { - size_t optimizations_size = passes.size(); - if (up_to_pass_index > optimizations_size) + size_t passes_size = passes.size(); + if (up_to_pass_index > passes_size) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Requested to dump optimizations up to {} pass. There are only {} pass", + "Requested to dump passes up to {} pass. There are only {} passes", up_to_pass_index, - optimizations_size); + passes_size); for (size_t i = 0; i < up_to_pass_index; ++i) { auto & pass = passes[i]; buffer << "Pass " << (i + 1) << " " << pass->getName() << " - " << pass->getDescription(); - if (i < up_to_pass_index) + if (i + 1 != up_to_pass_index) buffer << '\n'; } } diff --git a/src/Analyzer/QueryTreePassManager.h b/src/Analyzer/QueryTreePassManager.h index 0360d0bcae9..3c67fc36178 100644 --- a/src/Analyzer/QueryTreePassManager.h +++ b/src/Analyzer/QueryTreePassManager.h @@ -28,7 +28,7 @@ public: void run(QueryTreeNodePtr query_tree_node); /** Run query tree passes on query tree up to up_to_pass_index. - * If up_to_pass_index is greater than passes size throws exception. + * Throws exception if up_to_pass_index is greater than passes size. */ void run(QueryTreeNodePtr query_tree_node, size_t up_to_pass_index); @@ -36,7 +36,7 @@ public: void dump(WriteBuffer & buffer); /** Dump query tree passes to up_to_pass_index. - * If up_to_pass_index is greater than passes size throws exception. + * Throws exception if up_to_pass_index is greater than passes size. */ void dump(WriteBuffer & buffer, size_t up_to_pass_index); diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 9d33d936c39..a72879d2145 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -33,14 +33,14 @@ size_t getCompoundTypeDepth(const IDataType & type) if (which_type.isArray()) { - current_type = &(*assert_cast(*current_type).getNestedType()); + current_type = assert_cast(*current_type).getNestedType().get(); ++result; } else if (which_type.isTuple()) { const auto & tuple_elements = assert_cast(*current_type).getElements(); if (!tuple_elements.empty()) - current_type = &(*assert_cast(*current_type).getElements().at(0)); + current_type = tuple_elements.at(0).get(); ++result; } @@ -89,8 +89,8 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & if (tuple_size != columns_size) throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET, "Incorrect size of tuple in set: {} instead of {}", - toString(tuple_size), - toString(columns_size)); + tuple_size, + columns_size); if (tuple_values.empty()) tuple_values.resize(tuple_size); diff --git a/src/Analyzer/SortNode.cpp b/src/Analyzer/SortNode.cpp index 509a973c7d4..f6faccdb7c5 100644 --- a/src/Analyzer/SortNode.cpp +++ b/src/Analyzer/SortNode.cpp @@ -149,8 +149,6 @@ ASTPtr SortNode::toASTImpl() const result->nulls_direction = *nulls_sort_direction == SortDirection::ASCENDING ? 1 : -1; result->nulls_direction_was_explicitly_specified = nulls_sort_direction.has_value(); - if (collator) - result->collation = std::make_shared(Field(collator->getLocale())); result->with_fill = with_fill; result->fill_from = hasFillFrom() ? getFillFrom()->toAST() : nullptr; @@ -158,6 +156,12 @@ ASTPtr SortNode::toASTImpl() const result->fill_step = hasFillStep() ? getFillStep()->toAST() : nullptr; result->children.push_back(getExpression()->toAST()); + if (collator) + { + result->children.push_back(std::make_shared(Field(collator->getLocale()))); + result->collation = result->children.back(); + } + return result; } diff --git a/src/Analyzer/SortNode.h b/src/Analyzer/SortNode.h index a3c722db698..b0fe835cf45 100644 --- a/src/Analyzer/SortNode.h +++ b/src/Analyzer/SortNode.h @@ -45,13 +45,13 @@ public: return children[sort_expression_child_index]; } - /// Has with fill + /// Returns true if sort node has with fill, false otherwise bool withFill() const { return with_fill; } - /// Has fill from + /// Returns true if sort node has fill from, false otherwise bool hasFillFrom() const { return children[fill_from_child_index] != nullptr; @@ -69,7 +69,7 @@ public: return children[fill_from_child_index]; } - /// Has fill to + /// Returns true if sort node has fill to, false otherwise bool hasFillTo() const { return children[fill_to_child_index] != nullptr; @@ -87,6 +87,7 @@ public: return children[fill_to_child_index]; } + /// Returns true if sort node has fill step, false otherwise bool hasFillStep() const { return children[fill_step_child_index] != nullptr; @@ -98,7 +99,7 @@ public: return children[fill_step_child_index]; } - /// Get fill to + /// Get fill step QueryTreeNodePtr & getFillStep() { return children[fill_step_child_index]; @@ -110,11 +111,13 @@ public: return collator; } + /// Get sort direction SortDirection getSortDirection() const { return sort_direction; } + /// Get nulls sort direction std::optional getNullsSortDirection() const { return nulls_sort_direction; diff --git a/src/Analyzer/TableExpressionModifiers.h b/src/Analyzer/TableExpressionModifiers.h index ea66d62dec4..cc5ac3948bf 100644 --- a/src/Analyzer/TableExpressionModifiers.h +++ b/src/Analyzer/TableExpressionModifiers.h @@ -22,37 +22,44 @@ public: , sample_offset_ratio(sample_offset_ratio_) {} + /// Returns true if final is specified, false otherwise bool hasFinal() const { return has_final; } + /// Returns true if sample size ratio is specified, false otherwise bool hasSampleSizeRatio() const { return sample_size_ratio.has_value(); } + /// Get sample size ratio std::optional getSampleSizeRatio() const { return sample_size_ratio; } + /// Returns true if sample offset ratio is specified, false otherwise bool hasSampleOffsetRatio() const { return sample_offset_ratio.has_value(); } + /// Get sample offset ratio std::optional getSampleOffsetRatio() const { return sample_offset_ratio; } + /// Dump into buffer void dump(WriteBuffer & buffer) const; + /// Update tree hash void updateTreeHash(SipHash & hash_state) const; private: - bool has_final; + bool has_final = false; std::optional sample_size_ratio; std::optional sample_offset_ratio; }; diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index 6dc6548cd1a..a9f08ed4d1a 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -19,9 +19,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -class ITableFunction; -using TableFunctionPtr = std::shared_ptr; - /** Table function node represents table function in query tree. * Example: SELECT a FROM table_function(arguments...). * @@ -29,6 +26,9 @@ using TableFunctionPtr = std::shared_ptr; * * Table function resolution must be done during query analysis pass. */ +class ITableFunction; +using TableFunctionPtr = std::shared_ptr; + class TableFunctionNode; using TableFunctionNodePtr = std::shared_ptr; @@ -86,6 +86,7 @@ public: return storage; } + /// Get storage, throws exception if table function node is not resolved const StoragePtr & getStorageOrThrow() const { if (!storage) @@ -94,7 +95,7 @@ public: return storage; } - /// Resolve table function with table_function, storage and context + /// Resolve table function with table function, storage and context void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context); /// Get storage id, throws exception if function node is not resolved @@ -110,7 +111,7 @@ public: } /// Get table expression modifiers - std::optional getTableExpressionModifiers() const + const std::optional & getTableExpressionModifiers() const { return table_expression_modifiers; } @@ -121,7 +122,6 @@ public: table_expression_modifiers = std::move(table_expression_modifiers_value); } - QueryTreeNodeType getNodeType() const override { return QueryTreeNodeType::TABLE_FUNCTION; diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index 54064428bff..f8e897378d6 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -60,7 +60,7 @@ public: } /// Get table expression modifiers - std::optional getTableExpressionModifiers() const + const std::optional & getTableExpressionModifiers() const { return table_expression_modifiers; } diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 703c68316a7..fe913373981 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -50,15 +50,9 @@ NamesAndTypes UnionNode::computeProjectionColumns() const for (const auto & query_node : query_nodes) { if (auto * query_node_typed = query_node->as()) - { - auto projection_columns = query_node_typed->getProjectionColumns(); - query_node_projection = NamesAndTypes(projection_columns.begin(), projection_columns.end()); - } + query_node_projection = query_node_typed->getProjectionColumns(); else if (auto * union_node_typed = query_node->as()) - { - auto projection_columns = union_node_typed->computeProjectionColumns(); - query_node_projection = NamesAndTypes(projection_columns.begin(), projection_columns.end()); - } + query_node_projection = union_node_typed->computeProjectionColumns(); projections.push_back(query_node_projection); @@ -132,8 +126,11 @@ void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s if (hasAlias()) buffer << ", alias: " << getAlias(); - buffer << ", is_subquery: " << is_subquery; - buffer << ", is_cte: " << is_cte; + if (is_subquery) + buffer << ", is_subquery: " << is_subquery; + + if (is_cte) + buffer << ", is_cte: " << is_cte; if (!cte_name.empty()) buffer << ", cte_name: " << cte_name; @@ -187,7 +184,8 @@ bool UnionNode::isEqualImpl(const IQueryTreeNode & rhs) const else if (!table_expression_modifiers && rhs_typed.table_expression_modifiers) return false; - return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name; + return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name && + union_mode == rhs_typed.union_mode && union_modes == rhs_typed.union_modes; } void UnionNode::updateTreeHashImpl(HashState & state) const @@ -198,6 +196,12 @@ void UnionNode::updateTreeHashImpl(HashState & state) const state.update(cte_name.size()); state.update(cte_name); + state.update(static_cast(union_mode)); + + state.update(union_modes.size()); + for (const auto & query_union_mode : union_modes) + state.update(static_cast(query_union_mode)); + if (constant_value) { auto constant_dump = applyVisitor(FieldVisitorToString(), constant_value->getValue()); @@ -215,18 +219,18 @@ void UnionNode::updateTreeHashImpl(HashState & state) const QueryTreeNodePtr UnionNode::cloneImpl() const { - auto result_query_node = std::make_shared(); + auto result_union_node = std::make_shared(); - result_query_node->is_subquery = is_subquery; - result_query_node->is_cte = is_cte; - result_query_node->cte_name = cte_name; - result_query_node->union_mode = union_mode; - result_query_node->union_modes = union_modes; - result_query_node->union_modes_set = union_modes_set; - result_query_node->constant_value = constant_value; - result_query_node->table_expression_modifiers = table_expression_modifiers; + result_union_node->is_subquery = is_subquery; + result_union_node->is_cte = is_cte; + result_union_node->cte_name = cte_name; + result_union_node->union_mode = union_mode; + result_union_node->union_modes = union_modes; + result_union_node->union_modes_set = union_modes_set; + result_union_node->constant_value = constant_value; + result_union_node->table_expression_modifiers = table_expression_modifiers; - return result_query_node; + return result_union_node; } ASTPtr UnionNode::toASTImpl() const diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h index 8fe14871fdd..9c502c8fe5b 100644 --- a/src/Analyzer/UnionNode.h +++ b/src/Analyzer/UnionNode.h @@ -27,6 +27,12 @@ namespace ErrorCodes * Example: (SELECT id FROM test_table) INTERSECT ALL (SELECT id FROM test_table_2); * Example: (SELECT id FROM test_table) INTERSECT DISTINCT (SELECT id FROM test_table_2); * + * Union node can be used as CTE. + * Example: WITH cte_subquery AS ((SELECT id FROM test_table) UNION ALL (SELECT id FROM test_table_2)) SELECT * FROM cte_subquery; + * + * Union node can be used as scalar subquery. + * Example: SELECT (SELECT 1 UNION DISTINCT SELECT 1); + * * During query analysis pass union node queries must be resolved. */ class UnionNode; @@ -129,7 +135,7 @@ public: } /// Get table expression modifiers - std::optional getTableExpressionModifiers() const + const std::optional & getTableExpressionModifiers() const { return table_expression_modifiers; } diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index adce27902a7..5f0d682865f 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -132,11 +132,11 @@ static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expre { result_table_expression->final = table_expression_modifiers->hasFinal(); - auto sample_size_ratio = table_expression_modifiers->getSampleSizeRatio(); + const auto & sample_size_ratio = table_expression_modifiers->getSampleSizeRatio(); if (sample_size_ratio.has_value()) result_table_expression->sample_size = std::make_shared(*sample_size_ratio); - auto sample_offset_ratio = table_expression_modifiers->getSampleOffsetRatio(); + const auto & sample_offset_ratio = table_expression_modifiers->getSampleOffsetRatio(); if (sample_offset_ratio.has_value()) result_table_expression->sample_offset = std::make_shared(*sample_offset_ratio); } @@ -175,14 +175,14 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q { auto table_expression_tables_in_select_query_ast = table_expression->toAST(); tables_in_select_query_ast->children.reserve(table_expression_tables_in_select_query_ast->children.size()); - for (auto && left_table_element_ast : table_expression_tables_in_select_query_ast->children) - tables_in_select_query_ast->children.push_back(std::move(left_table_element_ast)); + for (auto && table_element_ast : table_expression_tables_in_select_query_ast->children) + tables_in_select_query_ast->children.push_back(std::move(table_element_ast)); break; } default: { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected node type for table expression. Expected table, table function, query, union, join or array join. Actual {}", + "Unexpected node type for table expression. Expected identifier, table, table function, query, union, join or array join. Actual {}", table_expression->getNodeTypeName()); } } diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 359809b4bde..325a7d2fcc8 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -19,7 +19,8 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q /// Extract table, table function, query, union from join tree QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node); -/** Build table expressions stack that consists from table, table function, query, union, join, array join from join tree +/** Build table expressions stack that consists from table, table function, query, union, join, array join from join tree. + * * Example: SELECT * FROM t1 INNER JOIN t2 INNER JOIN t3. * Result table expressions stack: * 1. t1 INNER JOIN t2 INNER JOIN t3 diff --git a/src/Analyzer/WindowFunctionsUtils.cpp b/src/Analyzer/WindowFunctionsUtils.cpp index e61dc4bf41d..fb411f2418c 100644 --- a/src/Analyzer/WindowFunctionsUtils.cpp +++ b/src/Analyzer/WindowFunctionsUtils.cpp @@ -35,7 +35,7 @@ public: if (!assert_no_window_functions_place_message.empty()) throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, "Window function {} is found {} in query", - function_node->getName(), + function_node->formatASTForErrorMessage(), assert_no_window_functions_place_message); if (window_function_nodes) diff --git a/src/Analyzer/WindowNode.cpp b/src/Analyzer/WindowNode.cpp index 4b92054b103..ccf49bc37d8 100644 --- a/src/Analyzer/WindowNode.cpp +++ b/src/Analyzer/WindowNode.cpp @@ -164,6 +164,8 @@ void WindowNode::updateTreeHashImpl(HashState & hash_state) const hash_state.update(window_frame.begin_preceding); hash_state.update(window_frame.end_type); hash_state.update(window_frame.end_preceding); + + hash_state.update(parent_window_name); } QueryTreeNodePtr WindowNode::cloneImpl() const @@ -178,6 +180,8 @@ ASTPtr WindowNode::toASTImpl() const { auto window_definition = std::make_shared(); + window_definition->parent_window_name = parent_window_name; + window_definition->children.push_back(getPartitionByNode()->toAST()); window_definition->partition_by = window_definition->children.back(); diff --git a/src/Analyzer/WindowNode.h b/src/Analyzer/WindowNode.h index 57875aad710..2f99524eebd 100644 --- a/src/Analyzer/WindowNode.h +++ b/src/Analyzer/WindowNode.h @@ -47,7 +47,7 @@ public: return window_frame; } - /// Has parent window name + /// Returns true if window node has parent window name, false otherwise bool hasParentWindowName() const { return parent_window_name.empty(); @@ -65,7 +65,7 @@ public: parent_window_name = std::move(parent_window_name_value); } - /// Has order by + /// Returns true if window node has order by, false otherwise bool hasOrderBy() const { return !getOrderBy().getNodes().empty(); @@ -83,19 +83,19 @@ public: return children[order_by_child_index]->as(); } - /// Get order by + /// Get order by node const QueryTreeNodePtr & getOrderByNode() const { return children[order_by_child_index]; } - /// Get order by + /// Get order by node QueryTreeNodePtr & getOrderByNode() { return children[order_by_child_index]; } - /// Has partition by + /// Returns true if window node has partition by, false otherwise bool hasPartitionBy() const { return !getPartitionBy().getNodes().empty(); @@ -113,19 +113,19 @@ public: return children[partition_by_child_index]->as(); } - /// Get partition by + /// Get partition by node const QueryTreeNodePtr & getPartitionByNode() const { return children[partition_by_child_index]; } - /// Get partition by + /// Get partition by node QueryTreeNodePtr & getPartitionByNode() { return children[partition_by_child_index]; } - /// Has frame begin offset + /// Returns true if window node has FRAME begin offset, false otherwise bool hasFrameBeginOffset() const { return getFrameBeginOffsetNode() != nullptr; @@ -143,19 +143,19 @@ public: return children[frame_begin_offset_child_index]; } - /// Has frame end offset + /// Returns true if window node has FRAME end offset, false otherwise bool hasFrameEndOffset() const { return getFrameEndOffsetNode() != nullptr; } - /// Get FRAME begin offset node + /// Get FRAME end offset node const QueryTreeNodePtr & getFrameEndOffsetNode() const { return children[frame_end_offset_child_index]; } - /// Get FRAME begin offset node + /// Get FRAME end offset node QueryTreeNodePtr & getFrameEndOffsetNode() { return children[frame_end_offset_child_index]; diff --git a/src/Parsers/ASTColumnsMatcher.cpp b/src/Parsers/ASTColumnsMatcher.cpp index a887b239902..c4f04f74a33 100644 --- a/src/Parsers/ASTColumnsMatcher.cpp +++ b/src/Parsers/ASTColumnsMatcher.cpp @@ -164,10 +164,11 @@ void ASTQualifiedColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state) void ASTQualifiedColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { + settings.ostr << (settings.hilite ? hilite_keyword : ""); + const auto & qualifier = children.at(0); qualifier->formatImpl(settings, state, frame); - settings.ostr << (settings.hilite ? hilite_keyword : ""); settings.ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "("; settings.ostr << quoteString(original_pattern); settings.ostr << ")"; @@ -212,10 +213,12 @@ void ASTQualifiedColumnsListMatcher::updateTreeHashImpl(SipHash & hash_state) co void ASTQualifiedColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { + settings.ostr << (settings.hilite ? hilite_keyword : ""); + const auto & qualifier = children.at(0); qualifier->formatImpl(settings, state, frame); - settings.ostr << (settings.hilite ? hilite_keyword : "") << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "("; + settings.ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "("; for (ASTs::const_iterator it = column_list->children.begin(); it != column_list->children.end(); ++it) { From 39b2d25a43911364ef3721e32e9d1fc685fe1dc2 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 19 Oct 2022 12:29:59 +0200 Subject: [PATCH 148/188] Added tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 15 ++++++++--- .../0_stateless/02387_analyzer_cte.reference | 7 +++++ .../0_stateless/02387_analyzer_cte.sql | 26 +++++++++++++++++++ 3 files changed, 44 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02387_analyzer_cte.reference create mode 100644 tests/queries/0_stateless/02387_analyzer_cte.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index afe89ce2696..21af0ace6ea 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5320,10 +5320,15 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier for (auto & node : with_nodes) { auto * subquery_node = node->as(); - if (!subquery_node || !subquery_node->isCTE()) + auto * union_node = node->as(); + + bool subquery_is_cte = (subquery_node && subquery_node->isCTE()) || (union_node && union_node->isCTE()); + + if (!subquery_is_cte) continue; - const auto & cte_name = subquery_node->getCTEName(); + const auto & cte_name = subquery_node ? subquery_node->getCTEName() : union_node->getCTEName(); + auto [_, inserted] = scope.cte_name_to_query_node.emplace(cte_name, node); if (!inserted) throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, @@ -5335,7 +5340,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier std::erase_if(with_nodes, [](const QueryTreeNodePtr & node) { auto * subquery_node = node->as(); - return subquery_node && subquery_node->isCTE(); + auto * union_node = node->as(); + + return (subquery_node && subquery_node->isCTE()) || (union_node && union_node->isCTE()); }); for (auto & window_node : query_node_typed.getWindow().getNodes()) @@ -5364,7 +5371,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier * * Example: SELECT id FROM test_table AS t1 INNER JOIN test_table AS t2 ON t1.id = t2.id INNER JOIN test_table AS t3 ON t1.id = t3.id * In first join expression ON t1.id = t2.id t1.id is resolved into test_table.id column. - * IN second join expression ON t1.id = t3.id t1.id must be resolved into join column that wrap test_table.id column. + * In second join expression ON t1.id = t3.id t1.id must be resolved into test_table.id column after first JOIN. */ scope.use_identifier_lookup_to_result_cache = false; diff --git a/tests/queries/0_stateless/02387_analyzer_cte.reference b/tests/queries/0_stateless/02387_analyzer_cte.reference new file mode 100644 index 00000000000..1ad3aee198b --- /dev/null +++ b/tests/queries/0_stateless/02387_analyzer_cte.reference @@ -0,0 +1,7 @@ +1 +-- +0 Value +-- +1 +-- +0 Value diff --git a/tests/queries/0_stateless/02387_analyzer_cte.sql b/tests/queries/0_stateless/02387_analyzer_cte.sql new file mode 100644 index 00000000000..725e18af315 --- /dev/null +++ b/tests/queries/0_stateless/02387_analyzer_cte.sql @@ -0,0 +1,26 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +WITH cte_subquery AS (SELECT 1) SELECT * FROM cte_subquery; + +SELECT '--'; + +WITH cte_subquery AS (SELECT * FROM test_table) SELECT * FROM cte_subquery; + +SELECT '--'; + +WITH cte_subquery AS (SELECT 1 UNION DISTINCT SELECT 1) SELECT * FROM cte_subquery; + +SELECT '--'; + +WITH cte_subquery AS (SELECT * FROM test_table UNION DISTINCT SELECT * FROM test_table) SELECT * FROM cte_subquery; + +DROP TABLE test_table; From 561d585100286084c57d48cb0f362f46743cb18f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 19 Oct 2022 12:46:10 +0200 Subject: [PATCH 149/188] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 39 ++++++++++++------- .../02388_analyzer_recursive_lambda.reference | 0 .../02388_analyzer_recursive_lambda.sql | 5 +++ 3 files changed, 29 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/02388_analyzer_recursive_lambda.reference create mode 100644 tests/queries/0_stateless/02388_analyzer_recursive_lambda.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 21af0ace6ea..4f9ddd48e81 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1110,7 +1110,10 @@ private: ProjectionName resolveWindow(QueryTreeNodePtr & window_node, IdentifierResolveScope & scope); - ProjectionNames resolveLambda(const QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope); + ProjectionNames resolveLambda(const QueryTreeNodePtr & lambda_node, + const QueryTreeNodePtr & lambda_node_to_resolve, + const QueryTreeNodes & lambda_arguments, + IdentifierResolveScope & scope); ProjectionNames resolveFunction(QueryTreeNodePtr & function_node, IdentifierResolveScope & scope); @@ -3103,7 +3106,7 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I { auto lambda_expression_to_resolve = expression_node->clone(); IdentifierResolveScope lambda_scope(expression_node, &scope /*parent_scope*/); - node_projection_names = resolveLambda(lambda_expression_to_resolve, {node}, lambda_scope); + node_projection_names = resolveLambda(expression_node, lambda_expression_to_resolve, {node}, lambda_scope); auto & lambda_expression_to_resolve_typed = lambda_expression_to_resolve->as(); node = lambda_expression_to_resolve_typed.getExpression(); } @@ -3398,21 +3401,25 @@ ProjectionName QueryAnalyzer::resolveWindow(QueryTreeNodePtr & node, IdentifierR * Lambda expression can be resolved into list node. It is caller responsibility to handle it properly. * * lambda_node - node that must have LambdaNode type. + * lambda_node_to_resolve - lambda node to resolve that must have LambdaNode type. * arguments - lambda arguments. * scope - lambda scope. It is client responsibility to create it. * * Resolve steps: * 1. Validate arguments. - * 2. Register lambda in lambdas in resolve process. This is necessary to prevent recursive lambda resolving. + * 2. Register lambda node in lambdas in resolve process. This is necessary to prevent recursive lambda resolving. * 3. Initialize scope with lambda aliases. * 4. Validate lambda argument names, and scope expressions. * 5. Resolve lambda body expression. - * 6. Deregister lambda from lambdas in resolve process. + * 6. Deregister lambda node from lambdas in resolve process. */ -ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const QueryTreeNodes & lambda_arguments, IdentifierResolveScope & scope) +ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, + const QueryTreeNodePtr & lambda_node_to_resolve, + const QueryTreeNodes & lambda_arguments, + IdentifierResolveScope & scope) { - auto & lambda = lambda_node->as(); - auto & lambda_arguments_nodes = lambda.getArguments().getNodes(); + auto & lambda_to_resolve = lambda_node_to_resolve->as(); + auto & lambda_arguments_nodes = lambda_to_resolve.getArguments().getNodes(); size_t lambda_arguments_nodes_size = lambda_arguments_nodes.size(); /** Register lambda as being resolved, to prevent recursive lambdas resolution. @@ -3422,21 +3429,22 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod if (it != lambdas_in_resolve_process.end()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Recursive lambda {}. In scope {}", - lambda.formatASTForErrorMessage(), + lambda_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); + lambdas_in_resolve_process.emplace(lambda_node.get()); size_t arguments_size = lambda_arguments.size(); if (lambda_arguments_nodes_size != arguments_size) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Lambda {} expect {} arguments. Actual {}. In scope {}", - lambda.formatASTForErrorMessage(), + lambda_to_resolve.formatASTForErrorMessage(), arguments_size, lambda_arguments_nodes_size, scope.scope_node->formatASTForErrorMessage()); /// Initialize aliases in lambda scope QueryExpressionsAliasVisitor visitor(scope); - visitor.visit(lambda.getExpression()); + visitor.visit(lambda_to_resolve.getExpression()); /** Replace lambda arguments with new arguments. * Additionally validate that there are no aliases with same name as lambda arguments. @@ -3467,10 +3475,10 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod lambda_new_arguments_nodes.push_back(lambda_arguments[i]); } - lambda.getArguments().getNodes() = std::move(lambda_new_arguments_nodes); + lambda_to_resolve.getArguments().getNodes() = std::move(lambda_new_arguments_nodes); /// Lambda body expression is resolved as standard query expression node. - auto result_projection_names = resolveExpressionNode(lambda.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + auto result_projection_names = resolveExpressionNode(lambda_to_resolve.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); lambdas_in_resolve_process.erase(lambda_node.get()); @@ -3770,7 +3778,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto lambda_expression_clone = lambda_expression_untyped->clone(); IdentifierResolveScope lambda_scope(lambda_expression_clone, &scope /*parent_scope*/); - ProjectionNames lambda_projection_names = resolveLambda(lambda_expression_clone, function_arguments, lambda_scope); + ProjectionNames lambda_projection_names = resolveLambda(lambda_expression_untyped, lambda_expression_clone, function_arguments, lambda_scope); auto & resolved_lambda = lambda_expression_clone->as(); node = resolved_lambda.getExpression(); @@ -3906,7 +3914,8 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi ProjectionNames lambda_projection_names; for (auto & function_lambda_argument_index : function_lambda_arguments_indexes) { - auto lambda_to_resolve = function_arguments[function_lambda_argument_index]->clone(); + auto & lambda_argument = function_arguments[function_lambda_argument_index]; + auto lambda_to_resolve = lambda_argument->clone(); auto & lambda_to_resolve_typed = lambda_to_resolve->as(); const auto & lambda_argument_names = lambda_to_resolve_typed.getArgumentNames(); @@ -3943,7 +3952,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } IdentifierResolveScope lambda_scope(lambda_to_resolve, &scope /*parent_scope*/); - lambda_projection_names = resolveLambda(lambda_to_resolve, lambda_arguments, lambda_scope); + lambda_projection_names = resolveLambda(lambda_argument, lambda_to_resolve, lambda_arguments, lambda_scope); if (auto * lambda_list_node_result = lambda_to_resolve_typed.getExpression()->as()) { diff --git a/tests/queries/0_stateless/02388_analyzer_recursive_lambda.reference b/tests/queries/0_stateless/02388_analyzer_recursive_lambda.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02388_analyzer_recursive_lambda.sql b/tests/queries/0_stateless/02388_analyzer_recursive_lambda.sql new file mode 100644 index 00000000000..dd342b32d1f --- /dev/null +++ b/tests/queries/0_stateless/02388_analyzer_recursive_lambda.sql @@ -0,0 +1,5 @@ +SET use_analyzer = 1; + +WITH x -> plus(lambda(1), x) AS lambda SELECT lambda(1048576); -- { serverError 1 }; + +WITH lambda(lambda(plus(x, x, -1)), tuple(x), x + 2147483646) AS lambda, x -> plus(lambda(1), x, 2) AS lambda SELECT 1048576, lambda(1048576); -- { serverError 1 }; From d083200d6584a919324b563893e419f9fa4d22bf Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 19 Oct 2022 18:49:17 +0200 Subject: [PATCH 150/188] Fixed code review issues --- src/Analyzer/ColumnTransformers.cpp | 6 +- ...egateFunctionsArithmericOperationsPass.cpp | 4 +- src/Analyzer/Passes/CountDistinctPass.cpp | 5 +- .../Passes/CustomizeFunctionsPass.cpp | 11 ++- .../Passes/FunctionToSubcolumnsPass.cpp | 56 ++++---------- .../Passes/IfConstantConditionPass.cpp | 2 +- src/Analyzer/Passes/IfConstantConditionPass.h | 1 + src/Analyzer/Passes/MultiIfToIfPass.cpp | 2 +- src/Analyzer/Passes/MultiIfToIfPass.h | 5 +- .../Passes/NormalizeCountVariantsPass.cpp | 2 +- .../Passes/NormalizeCountVariantsPass.h | 2 +- ...OrderByLimitByDuplicateEliminationPass.cpp | 4 +- .../OrderByLimitByDuplicateEliminationPass.h | 1 + .../Passes/OrderByTupleEliminationPass.cpp | 2 +- .../Passes/OrderByTupleEliminationPass.h | 1 + src/Analyzer/Passes/QueryAnalysisPass.h | 19 +++-- src/Analyzer/Passes/SumIfToCountIfPass.cpp | 19 ++--- .../UniqInjectiveFunctionsEliminationPass.cpp | 4 +- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- src/Parsers/ASTColumnsMatcher.cpp | 13 +++- src/Parsers/ParserExplainQuery.cpp | 2 - src/Planner/CMakeLists.txt | 2 +- src/Planner/PlannerJoinTree.cpp | 77 ++++++++++--------- src/Planner/PlannerJoins.cpp | 28 ++++--- src/Planner/PlannerJoins.h | 43 ++++++----- src/Planner/PlannerSorting.cpp | 7 +- src/Planner/PlannerWindowFunctions.cpp | 6 +- src/Planner/PlannerWindowFunctions.h | 4 +- src/Planner/TableExpressionData.h | 22 +++--- src/Planner/Utils.cpp | 32 ++++---- src/Planner/Utils.h | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 8 +- 32 files changed, 196 insertions(+), 198 deletions(-) diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp index e8c9ccb1a6d..27466ce5c27 100644 --- a/src/Analyzer/ColumnTransformers.cpp +++ b/src/Analyzer/ColumnTransformers.cpp @@ -285,7 +285,7 @@ void ReplaceColumnTransformerNode::dumpTreeImpl(WriteBuffer & buffer, FormatStat { buffer << std::string(indent, ' ') << "REPLACE COLUMN TRANSFORMER id: " << format_state.getNodeId(this); - auto & replacements_nodes = getReplacements().getNodes(); + const auto & replacements_nodes = getReplacements().getNodes(); size_t replacements_size = replacements_nodes.size(); buffer << '\n' << std::string(indent + 2, ' ') << "REPLACEMENTS " << replacements_size << '\n'; @@ -312,7 +312,7 @@ void ReplaceColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState { hash_state.update(static_cast(getTransformerType())); - auto & replacement_expressions_nodes = getReplacements().getNodes(); + const auto & replacement_expressions_nodes = getReplacements().getNodes(); size_t replacements_size = replacement_expressions_nodes.size(); hash_state.update(replacements_size); @@ -338,7 +338,7 @@ ASTPtr ReplaceColumnTransformerNode::toASTImpl() const { auto ast_replace_transformer = std::make_shared(); - auto & replacement_expressions_nodes = getReplacements().getNodes(); + const auto & replacement_expressions_nodes = getReplacements().getNodes(); size_t replacements_size = replacement_expressions_nodes.size(); ast_replace_transformer->children.reserve(replacements_size); diff --git a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp index 3be4c8d0bfa..dcf386b2988 100644 --- a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp +++ b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp @@ -47,12 +47,12 @@ class AggregateFunctionsArithmericOperationsVisitor : public InDepthQueryTreeVis { public: /// Traverse tree bottom to top - bool shouldTraverseTopToBottom() const + static bool shouldTraverseTopToBottom() { return false; } - void visitImpl(QueryTreeNodePtr & node) + static void visitImpl(QueryTreeNodePtr & node) { auto * aggregate_function_node = node->as(); if (!aggregate_function_node || !aggregate_function_node->isAggregateFunction()) diff --git a/src/Analyzer/Passes/CountDistinctPass.cpp b/src/Analyzer/Passes/CountDistinctPass.cpp index 60d44e13e7f..2b55efa3552 100644 --- a/src/Analyzer/Passes/CountDistinctPass.cpp +++ b/src/Analyzer/Passes/CountDistinctPass.cpp @@ -33,7 +33,7 @@ public: return; /// Check that query has only single node in projection - auto projection_nodes = query_node->getProjection().getNodes(); + auto & projection_nodes = query_node->getProjection().getNodes(); if (projection_nodes.size() != 1) return; @@ -43,7 +43,8 @@ public: if (!function_node) return; - if (Poco::toLower(function_node->getFunctionName()) != "countdistinct" && Poco::toLower(function_node->getFunctionName()) != "uniqexact") + auto lower_function_name = Poco::toLower(function_node->getFunctionName()); + if (lower_function_name != "countdistinct" && lower_function_name != "uniqexact") return; /// Check that `countDistinct` function has single COLUMN argument diff --git a/src/Analyzer/Passes/CustomizeFunctionsPass.cpp b/src/Analyzer/Passes/CustomizeFunctionsPass.cpp index 3c207bc35a1..629ab411a55 100644 --- a/src/Analyzer/Passes/CustomizeFunctionsPass.cpp +++ b/src/Analyzer/Passes/CustomizeFunctionsPass.cpp @@ -47,7 +47,7 @@ public: function_name_lowercase = Poco::toLower(function_name); } - /// Replace countDistinct with countIfDistinct with countDistinctIf implementation + /// Replace countIfDistinct with countDistinctIf implementation if (function_name_lowercase == "countifdistinct") { resolveAggregateOrWindowFunctionNode(*function_node, count_distinct_implementation_function_name + "If"); @@ -55,7 +55,7 @@ public: function_name_lowercase = Poco::toLower(function_name); } - /// Swap aggregateFunctionIfDistinct into aggregateFunctionDistinctIf to make execution more optimal + /// Replace aggregateFunctionIfDistinct into aggregateFunctionDistinctIf to make execution more optimal if (function_name_lowercase.ends_with("ifdistinct")) { size_t prefix_length = function_name_lowercase.size() - strlen("ifdistinct"); @@ -93,10 +93,13 @@ public: static constexpr std::array suffixes_to_replace = {"MergeState", "Merge", "State", "If"}; for (const auto & suffix : suffixes_to_replace) { - if (!function_name_lowercase.ends_with(suffix)) + auto suffix_string_value = String(suffix); + auto suffix_to_check = suffix_string_value + "OrNull"; + + if (!function_name.ends_with(suffix_to_check)) continue; - auto updated_function_name = function_name_lowercase.substr(0, function_name_size - suffix.size()) + "OrNull" + String(suffix); + auto updated_function_name = function_name.substr(0, function_name_size - suffix_to_check.size()) + "OrNull" + suffix_string_value; resolveAggregateOrWindowFunctionNode(*function_node, updated_function_name); function_name = function_node->getFunctionName(); function_name_lowercase = Poco::toLower(function_name); diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index a112d2e6886..41cc7bf18b1 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -35,7 +35,7 @@ public: if (!function_node) return; - auto function_arguments_nodes = function_node->getArguments().getNodes(); + auto & function_arguments_nodes = function_node->getArguments().getNodes(); size_t function_arguments_nodes_size = function_arguments_nodes.size(); if (function_arguments_nodes.empty() || function_arguments_nodes_size > 2) @@ -67,7 +67,7 @@ public: { if (function_name == "length") { - /// Replace `length(array_argument)` with `array_argument.length` + /// Replace `length(array_argument)` with `array_argument.size0` column.name += ".size0"; node = std::make_shared(column, column_source); @@ -78,15 +78,11 @@ public: column.name += ".size0"; column.type = std::make_shared(); - auto equals_function = std::make_shared("equals"); - resolveOrdinaryFunctionNode(*equals_function, "equals"); + resolveOrdinaryFunctionNode(*function_node, "equals"); - auto & equals_function_arguments = equals_function->getArguments().getNodes(); - equals_function_arguments.reserve(2); - equals_function_arguments.push_back(std::make_shared(column, column_source)); - equals_function_arguments.push_back(std::make_shared(static_cast(0))); - - node = std::move(equals_function); + function_arguments_nodes.clear(); + function_arguments_nodes.push_back(std::make_shared(column, column_source)); + function_arguments_nodes.push_back(std::make_shared(static_cast(0))); } else if (function_name == "notEmpty") { @@ -94,15 +90,11 @@ public: column.name += ".size0"; column.type = std::make_shared(); - auto not_equals_function = std::make_shared("notEquals"); - resolveOrdinaryFunctionNode(*not_equals_function, "notEquals"); + resolveOrdinaryFunctionNode(*function_node, "notEquals"); - auto & not_equals_function_arguments = not_equals_function->getArguments().getNodes(); - not_equals_function_arguments.reserve(2); - not_equals_function_arguments.push_back(std::make_shared(column, column_source)); - not_equals_function_arguments.push_back(std::make_shared(static_cast(0))); - - node = std::move(not_equals_function); + function_arguments_nodes.clear(); + function_arguments_nodes.push_back(std::make_shared(column, column_source)); + function_arguments_nodes.push_back(std::make_shared(static_cast(0))); } } else if (column_type.isNullable()) @@ -120,24 +112,18 @@ public: column.name += ".null"; column.type = std::make_shared(); - auto not_function = std::make_shared("not"); - resolveOrdinaryFunctionNode(*not_function, "not"); + resolveOrdinaryFunctionNode(*function_node, "not"); - auto & not_function_arguments = not_function->getArguments().getNodes(); - not_function_arguments.push_back(std::make_shared(column, column_source)); - - node = std::move(not_function); + function_arguments_nodes = {std::make_shared(column, column_source)}; } } else if (column_type.isMap()) { - const auto & data_type_map = assert_cast(*column.type); - if (function_name == "mapKeys") { /// Replace `mapKeys(map_argument)` with `map_argument.keys` column.name += ".keys"; - column.type = data_type_map.getKeyType(); + column.type = function_node->getResultType(); node = std::make_shared(column, column_source); } @@ -145,7 +131,7 @@ public: { /// Replace `mapValues(map_argument)` with `map_argument.values` column.name += ".values"; - column.type = data_type_map.getValueType(); + column.type = function_node->getResultType(); node = std::make_shared(column, column_source); } @@ -183,9 +169,7 @@ public: column.name += '.'; column.name += subcolumn_name; - - size_t subcolumn_position = data_type_tuple.getPositionByName(subcolumn_name); - column.type = data_type_tuple.getElement(subcolumn_position); + column.type = function_node->getResultType(); node = std::make_shared(column, column_source); } @@ -198,15 +182,9 @@ public: column.type = data_type_map.getKeyType(); auto has_function_argument = std::make_shared(column, column_source); - auto has_function = std::make_shared("has"); - resolveOrdinaryFunctionNode(*has_function, "has"); + resolveOrdinaryFunctionNode(*function_node, "has"); - auto & has_function_arguments = has_function->getArguments().getNodes(); - has_function_arguments.reserve(2); - has_function_arguments.push_back(std::move(has_function_argument)); - has_function_arguments.push_back(std::move(function_arguments_nodes[1])); - - node = std::move(has_function); + function_arguments_nodes[0] = std::move(has_function_argument); } } } diff --git a/src/Analyzer/Passes/IfConstantConditionPass.cpp b/src/Analyzer/Passes/IfConstantConditionPass.cpp index d114819ce86..1da1f5bd471 100644 --- a/src/Analyzer/Passes/IfConstantConditionPass.cpp +++ b/src/Analyzer/Passes/IfConstantConditionPass.cpp @@ -13,7 +13,7 @@ namespace class IfConstantConditionVisitor : public InDepthQueryTreeVisitor { public: - void visitImpl(QueryTreeNodePtr & node) + static void visitImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || (function_node->getFunctionName() != "if" && function_node->getFunctionName() != "multiIf")) diff --git a/src/Analyzer/Passes/IfConstantConditionPass.h b/src/Analyzer/Passes/IfConstantConditionPass.h index 571a27346c8..7817e67aa5e 100644 --- a/src/Analyzer/Passes/IfConstantConditionPass.h +++ b/src/Analyzer/Passes/IfConstantConditionPass.h @@ -7,6 +7,7 @@ namespace DB /** Convert `if` with constant condition or `multiIf` with single constant condition into true condition argument value * or false condition argument value. + * * Example: SELECT if(1, true_value, false_value); * Result: SELECT true_value; * diff --git a/src/Analyzer/Passes/MultiIfToIfPass.cpp b/src/Analyzer/Passes/MultiIfToIfPass.cpp index 777ac3e90d2..6d2ebac33e6 100644 --- a/src/Analyzer/Passes/MultiIfToIfPass.cpp +++ b/src/Analyzer/Passes/MultiIfToIfPass.cpp @@ -27,7 +27,7 @@ public: return; auto result_type = function_node->getResultType(); - function_node->resolveAsFunction(if_function_ptr, result_type); + function_node->resolveAsFunction(if_function_ptr, std::move(result_type)); } private: diff --git a/src/Analyzer/Passes/MultiIfToIfPass.h b/src/Analyzer/Passes/MultiIfToIfPass.h index 6791e8ec463..2213f3713ed 100644 --- a/src/Analyzer/Passes/MultiIfToIfPass.h +++ b/src/Analyzer/Passes/MultiIfToIfPass.h @@ -5,7 +5,8 @@ namespace DB { -/** Convert `multiIf` with single argument into `if`. +/** Convert `multiIf` with single condition into `if`. + * * Example: SELECT multiIf(x, 1, 0); * Result: SELECT if(x, 1, 0); */ @@ -14,7 +15,7 @@ class MultiIfToIfPass final : public IQueryTreePass public: String getName() override { return "MultiIfToIf"; } - String getDescription() override { return "Optimize multiIf to if for single argument."; } + String getDescription() override { return "Optimize multiIf with single condition to if."; } void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; diff --git a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp index c7e70d4b848..8c92ecc3900 100644 --- a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp @@ -15,7 +15,7 @@ namespace class NormalizeCountVariantsVisitor : public InDepthQueryTreeVisitor { public: - void visitImpl(QueryTreeNodePtr & node) + static void visitImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || !function_node->isAggregateFunction() || (function_node->getFunctionName() != "count" && function_node->getFunctionName() != "sum")) diff --git a/src/Analyzer/Passes/NormalizeCountVariantsPass.h b/src/Analyzer/Passes/NormalizeCountVariantsPass.h index 974d96abd74..78a114f4a85 100644 --- a/src/Analyzer/Passes/NormalizeCountVariantsPass.h +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.h @@ -7,7 +7,7 @@ namespace DB /** Remove single literal argument from `count`. Convert `sum` with single `1` literal argument into `count`. * - * Example: SELECT count(1) + * Example: SELECT count(1); * Result: SELECT count(); * * Example: SELECT sum(1); diff --git a/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.cpp b/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.cpp index 6b84f376115..0232d8958ff 100644 --- a/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.cpp +++ b/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.cpp @@ -70,7 +70,7 @@ public: auto & query_limit_by_nodes = query_node->getLimitBy().getNodes(); - for (auto & limit_by_node : query_node->getLimitBy().getNodes()) + for (auto & limit_by_node : query_limit_by_nodes) { auto [_, inserted] = unique_expressions_nodes_set.emplace(limit_by_node.get()); if (inserted) @@ -79,8 +79,6 @@ public: query_limit_by_nodes = std::move(result_nodes); } - - unique_expressions_nodes_set.clear(); } private: diff --git a/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h b/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h index f0043a0bc78..11a025af5b9 100644 --- a/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h +++ b/src/Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h @@ -6,6 +6,7 @@ namespace DB { /** Eliminate duplicate columns from ORDER BY and LIMIT BY. + * * Example: SELECT * FROM test_table ORDER BY id, id; * Result: SELECT * FROM test_table ORDER BY id; * diff --git a/src/Analyzer/Passes/OrderByTupleEliminationPass.cpp b/src/Analyzer/Passes/OrderByTupleEliminationPass.cpp index 14ea7c31416..f70ec27ba5d 100644 --- a/src/Analyzer/Passes/OrderByTupleEliminationPass.cpp +++ b/src/Analyzer/Passes/OrderByTupleEliminationPass.cpp @@ -15,7 +15,7 @@ namespace class OrderByTupleEliminationVisitor : public InDepthQueryTreeVisitor { public: - void visitImpl(QueryTreeNodePtr & node) + static void visitImpl(QueryTreeNodePtr & node) { auto * query_node = node->as(); if (!query_node || !query_node->hasOrderBy()) diff --git a/src/Analyzer/Passes/OrderByTupleEliminationPass.h b/src/Analyzer/Passes/OrderByTupleEliminationPass.h index a25d52b8737..5665561e227 100644 --- a/src/Analyzer/Passes/OrderByTupleEliminationPass.h +++ b/src/Analyzer/Passes/OrderByTupleEliminationPass.h @@ -6,6 +6,7 @@ namespace DB { /** Eliminate tuples from ORDER BY. + * * Example: SELECT * FROM test_table ORDER BY (a, b); * Result: SELECT * FROM test_table ORDER BY a, b; */ diff --git a/src/Analyzer/Passes/QueryAnalysisPass.h b/src/Analyzer/Passes/QueryAnalysisPass.h index 3ae2713a2ee..677a13044f2 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.h +++ b/src/Analyzer/Passes/QueryAnalysisPass.h @@ -16,29 +16,34 @@ namespace DB * as aggregate or non aggregate function. * 4. All lambda expressions that are function arguments are resolved. Next passes can expect that LambaNode expression is resolved, and lambda has concrete arguments. * 5. All standalone lambda expressions are resolved. Next passes can expect that there will be no standalone LambaNode expressions in query. - * 6. Constants are folded. Example: SELECT plus(1, 1). After step will be: SELECT 2. + * 6. Constants are folded. Example: SELECT plus(1, 1). * Motivation for this, there are places in query tree that must contain constant: - * Function parameters Example: SELECT quantile(0.5)(x). - * Functions in which result type depends on constant expression. Example: cast(x, 'type_name'). - * Expressions that are part of LIMIT. Example: SELECT * FROM test_table LIMIT expr. + * Function parameters. Example: SELECT quantile(0.5)(x). + * Functions in which result type depends on constant expression argument. Example: cast(x, 'type_name'). + * Expressions that are part of LIMIT BY LIMIT, LIMIT BY OFFSET, LIMIT, OFFSET. Example: SELECT * FROM test_table LIMIT expr. + * Window function window frame OFFSET begin and OFFSET end. * * 7. All scalar subqueries are evaluated. * TODO: Scalar subqueries must be evaluated only if they are part of query tree where we must have constant. This is currently not done * because execution layer does not support scalar subqueries execution. * - * 8. For query node projection columns are calculated. Later passes cannot change type, display name of projection column, and cannot add or remove + * 8. For query node. + * + * Projection columns are calculated. Later passes cannot change type, display name of projection column, and cannot add or remove * columns in projection section. + * WITH and WINDOW sections are removed. * * 9. Query is validated. Parts that are validated: * * Constness of function parameters. * Constness of LIMIT and OFFSET. * Window functions frame. Constness of window functions frame begin OFFSET, end OFFSET. - * In SELECT, ORDER BY only columns that are specified in GROUP BY keys after GROUP BY are used. + * In query only columns that are specified in GROUP BY keys after GROUP BY are used. * GROUPING function arguments are specified in GROUP BY keys. * No GROUPING function if there is no GROUP BY. * No aggregate functions in JOIN TREE, WHERE, PREWHERE, GROUP BY and inside another aggregate functions. * GROUP BY modifiers CUBE, ROLLUP, GROUPING SETS and WITH TOTALS. + * Table expression modifiers are validated for table and table function nodes in JOIN TREE. * Table expression modifiers are disabled for subqueries in JOIN TREE. * For JOIN, ARRAY JOIN subqueries and table functions must have alias (Can be changed using joined_subquery_requires_alias setting). * @@ -53,7 +58,7 @@ namespace DB * * For function `in` and its variations arguments are resolved, but sets are not build. * If left and right arguments are constants constant folding is performed. - * If right argument resolved as table function, or table, and table is not of type Set, it is replaced with query that read only ordinary columns from underlying + * If right argument resolved as table, and table is not of type Set, it is replaced with query that read only ordinary columns from underlying * storage. * Example: SELECT id FROM test_table WHERE id IN test_table_other; * Result: SELECT id FROM test_table WHERE id IN (SELECT test_table_column FROM test_table_other); diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.cpp b/src/Analyzer/Passes/SumIfToCountIfPass.cpp index 8dde6d9f4d4..f43c90e10eb 100644 --- a/src/Analyzer/Passes/SumIfToCountIfPass.cpp +++ b/src/Analyzer/Passes/SumIfToCountIfPass.cpp @@ -1,5 +1,8 @@ #include +#include +#include + #include #include @@ -73,7 +76,7 @@ public: if (!nested_function || nested_function->getFunctionName() != "if") return; - auto nested_if_function_arguments_nodes = nested_function->getArguments().getNodes(); + auto & nested_if_function_arguments_nodes = nested_function->getArguments().getNodes(); if (nested_if_function_arguments_nodes.size() != 3) return; @@ -106,8 +109,13 @@ public: /// Rewrite `sum(if(cond, 0, 1))` into `countIf(not(cond))`. if (if_true_condition_value == 0 && if_false_condition_value == 1) { + auto condition_result_type = nested_if_function_arguments_nodes[0]->getResultType(); + DataTypePtr not_function_result_type = std::make_shared(); + if (condition_result_type->isNullable()) + not_function_result_type = makeNullable(not_function_result_type); + auto not_function = std::make_shared("not"); - resolveOrdinaryFunctionNode(*not_function, "not"); + not_function->resolveAsFunction(FunctionFactory::instance().get("not", context), std::move(not_function_result_type)); auto & not_function_arguments = not_function->getArguments().getNodes(); not_function_arguments.push_back(std::move(nested_if_function_arguments_nodes[0])); @@ -121,13 +129,6 @@ public: } private: - inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const - { - auto function_result_type = function_node.getResultType(); - auto function = FunctionFactory::instance().get(function_name, context); - function_node.resolveAsFunction(function, std::move(function_result_type)); - } - static inline void resolveAggregateFunctionNode(FunctionNode & function_node, const String & aggregate_function_name) { auto function_result_type = function_node.getResultType(); diff --git a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp index ed7b6d5c445..6520cb0717d 100644 --- a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp +++ b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp @@ -24,7 +24,7 @@ bool isUniqFunction(const String & function_name) class UniqInjectiveFunctionsEliminationVisitor : public InDepthQueryTreeVisitor { public: - void visitImpl(QueryTreeNodePtr & node) + static void visitImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || !function_node->isAggregateFunction() || !isUniqFunction(function_node->getFunctionName())) @@ -48,7 +48,7 @@ public: continue; /// Replace injective function with its single argument - uniq_function_argument_node = std::move(uniq_function_argument_node_argument_nodes[0]); + uniq_function_argument_node = uniq_function_argument_node_argument_nodes[0]; } } }; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index f3580902c3b..6ea1b6a62b3 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -374,7 +374,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() case ASTExplainQuery::QueryTree: { if (ast.getExplainedQuery()->as() == nullptr) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT is supported for EXPLAIN QUERYTREE query"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT is supported for EXPLAIN QUERY TREE query"); auto settings = checkAndGetSettings(ast.getSettings()); auto query_tree = buildQueryTree(ast.getExplainedQuery(), getContext()); diff --git a/src/Parsers/ASTColumnsMatcher.cpp b/src/Parsers/ASTColumnsMatcher.cpp index c4f04f74a33..06980208774 100644 --- a/src/Parsers/ASTColumnsMatcher.cpp +++ b/src/Parsers/ASTColumnsMatcher.cpp @@ -112,8 +112,11 @@ void ASTColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatSt settings.ostr << ")"; /// Format column transformers - for (const auto & child : children) + size_t children_size = children.size(); + + for (size_t i = 1; i < children_size; ++i) { + const auto & child = children[i]; settings.ostr << ' '; child->formatImpl(settings, state, frame); } @@ -174,8 +177,11 @@ void ASTQualifiedColumnsRegexpMatcher::formatImpl(const FormatSettings & setting settings.ostr << ")"; /// Format column transformers - for (const auto & child : children) + size_t children_size = children.size(); + + for (size_t i = 1; i < children_size; ++i) { + const auto & child = children[i]; settings.ostr << ' '; child->formatImpl(settings, state, frame); } @@ -223,9 +229,8 @@ void ASTQualifiedColumnsListMatcher::formatImpl(const FormatSettings & settings, for (ASTs::const_iterator it = column_list->children.begin(); it != column_list->children.end(); ++it) { if (it != column_list->children.begin()) - { settings.ostr << ", "; - } + (*it)->formatImpl(settings, state, frame); } settings.ostr << ")"; diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index 229cc94f6ff..7fc997f9548 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -92,9 +92,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (select_p.parse(pos, query, expected)) explain_query->setExplainedQuery(std::move(query)); else - { return false; - } } else if (kind == ASTExplainQuery::ExplainKind::CurrentTransaction) { diff --git a/src/Planner/CMakeLists.txt b/src/Planner/CMakeLists.txt index 1068fee4cea..766767b5c13 100644 --- a/src/Planner/CMakeLists.txt +++ b/src/Planner/CMakeLists.txt @@ -4,4 +4,4 @@ endif() if (ENABLE_EXAMPLES) add_subdirectory(examples) -endif() \ No newline at end of file +endif() diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index e5958284f2d..4cb446a65a0 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -111,29 +111,34 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, auto & query_context = planner_context->getQueryContext(); auto from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); - const auto & columns_names = table_expression_data.getColumnsNames(); - Names column_names(columns_names.begin(), columns_names.end()); + const auto & columns_names_set = table_expression_data.getColumnsNames(); + Names columns_names(columns_names_set.begin(), columns_names_set.end()); /** The current user must have the SELECT privilege. - * We do not check access rights for table functions because they have beein already checked in ITablefunction::execute(). + * We do not check access rights for table functions because they have been already checked in ITableFunction::execute(). */ if (table_node) - checkAccessRights(*table_node, column_names, planner_context->getQueryContext()); + { + auto column_names_with_aliases = columns_names; + const auto & alias_columns_names = table_expression_data.getAliasColumnsNames(); + column_names_with_aliases.insert(column_names_with_aliases.end(), alias_columns_names.begin(), alias_columns_names.end()); + checkAccessRights(*table_node, column_names_with_aliases, planner_context->getQueryContext()); + } - if (column_names.empty()) + if (columns_names.empty()) { auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); auto additional_column_to_read = column_names_and_types.front(); const auto & column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(additional_column_to_read, table_expression); - column_names.push_back(additional_column_to_read.name); + columns_names.push_back(additional_column_to_read.name); table_expression_data.addColumn(additional_column_to_read, column_identifier); } size_t max_block_size = query_context->getSettingsRef().max_block_size; size_t max_streams = query_context->getSettingsRef().max_threads; - bool need_rewrite_query_with_final = storage->needRewriteQueryWithFinal(column_names); + bool need_rewrite_query_with_final = storage->needRewriteQueryWithFinal(columns_names); if (need_rewrite_query_with_final) { if (table_expression_query_info.table_expression_modifiers) @@ -154,12 +159,12 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, } } - storage->read(query_plan, column_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); + storage->read(query_plan, columns_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); /// Create step which reads from empty source if storage has no data. if (!query_plan.isInitialized()) { - auto source_header = storage_snapshot->getSampleBlockForColumns(column_names); + auto source_header = storage_snapshot->getSampleBlockForColumns(columns_names); Pipe pipe(std::make_shared(source_header)); auto read_from_pipe = std::make_unique(std::move(pipe)); read_from_pipe->setStepDescription("Read from NullSource"); @@ -181,11 +186,15 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, auto rename_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); - for (const auto & [column_name, column_identifier] : table_expression_data.getColumnNameToIdentifier()) + for (auto & output_node : rename_actions_dag->getOutputs()) { - auto position = query_plan.getCurrentDataStream().header.getPositionByName(column_name); - const auto * node_to_rename = rename_actions_dag->getOutputs()[position]; - rename_actions_dag->getOutputs()[position] = &rename_actions_dag->addAlias(*node_to_rename, column_identifier); + const auto * column_identifier = table_expression_data.getColumnIdentifierOrNull(output_node->result_name); + + if (!column_identifier) + continue; + + const auto * node_to_rename = output_node; + output_node = &rename_actions_dag->addAlias(*node_to_rename, *column_identifier); } auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), rename_actions_dag); @@ -267,13 +276,13 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, const auto & join_node_using_column_node_type = join_node_using_column_node.getColumnType(); if (!left_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) { - auto left_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(left_inner_column_node); + const auto & left_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(left_inner_column_node); left_plan_column_name_to_cast_type.emplace(left_inner_column_identifier, join_node_using_column_node_type); } if (!right_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) { - auto right_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(right_inner_column_node); + const auto & right_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(right_inner_column_node); right_plan_column_name_to_cast_type.emplace(right_inner_column_identifier, join_node_using_column_node_type); } } @@ -468,18 +477,12 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, for (auto & join_using_node : using_list.getNodes()) { auto & join_using_column_node = join_using_node->as(); - if (!join_using_column_node.getExpression() || - join_using_column_node.getExpression()->getNodeType() != QueryTreeNodeType::LIST) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "JOIN {} column in USING does not have inner columns", - join_node.formatASTForErrorMessage()); - - auto & using_join_columns_list = join_using_column_node.getExpression()->as(); + auto & using_join_columns_list = join_using_column_node.getExpressionOrThrow()->as(); auto & using_join_left_join_column_node = using_join_columns_list.getNodes().at(0); auto & using_join_right_join_column_node = using_join_columns_list.getNodes().at(1); - auto left_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_left_join_column_node); - auto right_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_right_join_column_node); + const auto & left_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_left_join_column_node); + const auto & right_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(using_join_right_join_column_node); table_join_clause.key_names_left.push_back(left_column_identifier); table_join_clause.key_names_right.push_back(right_column_identifier); @@ -627,13 +630,13 @@ QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression, { auto & array_join_node = table_expression->as(); - auto left_plan = buildQueryPlanForJoinTreeNode(array_join_node.getTableExpression(), + auto plan = buildQueryPlanForJoinTreeNode(array_join_node.getTableExpression(), select_query_info, select_query_options, planner_context); - auto left_plan_output_columns = left_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + auto plan_output_columns = plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - ActionsDAGPtr array_join_action_dag = std::make_shared(left_plan_output_columns); + ActionsDAGPtr array_join_action_dag = std::make_shared(plan_output_columns); PlannerActionsVisitor actions_visitor(planner_context); NameSet array_join_columns; @@ -652,16 +655,16 @@ QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression, } array_join_action_dag->projectInput(); - auto array_join_actions = std::make_unique(left_plan.getCurrentDataStream(), array_join_action_dag); + auto array_join_actions = std::make_unique(plan.getCurrentDataStream(), array_join_action_dag); array_join_actions->setStepDescription("ARRAY JOIN actions"); - left_plan.addStep(std::move(array_join_actions)); + plan.addStep(std::move(array_join_actions)); auto array_join_action = std::make_shared(array_join_columns, array_join_node.isLeft(), planner_context->getQueryContext()); - auto array_join_step = std::make_unique(left_plan.getCurrentDataStream(), std::move(array_join_action)); + auto array_join_step = std::make_unique(plan.getCurrentDataStream(), std::move(array_join_action)); array_join_step->setStepDescription("ARRAY JOIN"); - left_plan.addStep(std::move(array_join_step)); + plan.addStep(std::move(array_join_step)); - return left_plan; + return plan; } } @@ -675,13 +678,13 @@ QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, switch (join_tree_node_type) { - case QueryTreeNodeType::QUERY: - [[fallthrough]]; - case QueryTreeNodeType::UNION: - [[fallthrough]]; case QueryTreeNodeType::TABLE: [[fallthrough]]; case QueryTreeNodeType::TABLE_FUNCTION: + [[fallthrough]]; + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::UNION: { return buildQueryPlanForTableExpression(join_tree_node, select_query_info, select_query_options, planner_context); } @@ -696,7 +699,7 @@ QueryPlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, default: { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected query, table, table function, join or array join query node. Actual {}", + "Expected table, table function, query, union, join or array join query node. Actual {}", join_tree_node->formatASTForErrorMessage()); } } diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 59c6e47a50d..e64de3359e8 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -75,7 +75,7 @@ void JoinClause::dump(WriteBuffer & buffer) const buffer << " left_condition_nodes: " + dump_dag_nodes(left_filter_condition_nodes); if (!right_filter_condition_nodes.empty()) - buffer << " left_condition_nodes: " + dump_dag_nodes(right_filter_condition_nodes); + buffer << " right_condition_nodes: " + dump_dag_nodes(right_filter_condition_nodes); } String JoinClause::dump() const @@ -190,7 +190,7 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "JOIN {} ON expression {} with constants is not supported", join_node.formatASTForErrorMessage(), - join_expressions_actions_node->function->getName()); + join_expressions_actions_node->result_name); } else if (left_expression_side_optional && !right_expression_side_optional) { @@ -210,7 +210,7 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, const ActionsDAG::Node * left_key = left_child; const ActionsDAG::Node * right_key = right_child; - if (left_expression_side_optional == JoinTableSide::Right) + if (left_expression_side == JoinTableSide::Right) { left_key = right_child; right_key = left_child; @@ -254,7 +254,6 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, join_node.formatASTForErrorMessage()); auto expression_side = *expression_side_optional; - join_clause.addCondition(expression_side, join_expressions_actions_node); } @@ -269,7 +268,7 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & /** In ActionsDAG if input node has constant representation additional constant column is added. * That way we cannot simply check that node has INPUT type during resolution of expression join table side. * Put all nodes after actions dag initialization in set. - * To check if actions dag node is input column, we set contains node. + * To check if actions dag node is input column, we check if set contains it. */ const auto & join_expression_actions_nodes = join_expression_actions->getNodes(); @@ -375,9 +374,7 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & else dag_filter_condition_node = left_filter_condition_nodes[0]; - join_clause.getLeftFilterConditionNodes().clear(); - join_clause.addCondition(JoinTableSide::Left, dag_filter_condition_node); - + join_clause.getLeftFilterConditionNodes() = {dag_filter_condition_node}; join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node); add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name); @@ -393,9 +390,7 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & else dag_filter_condition_node = right_filter_condition_nodes[0]; - join_clause.getRightFilterConditionNodes().clear(); - join_clause.addCondition(JoinTableSide::Right, dag_filter_condition_node); - + join_clause.getRightFilterConditionNodes() = {dag_filter_condition_node}; join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node); add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name); @@ -541,7 +536,8 @@ void trySetStorageInTableJoin(const QueryTreeNodePtr & table_expression, std::sh else if (auto * table_function = table_expression->as()) storage = table_function->getStorage(); - if (auto storage_join = std::dynamic_pointer_cast(storage); storage_join) + auto storage_join = std::dynamic_pointer_cast(storage); + if (storage_join) { table_join->setStorageJoin(storage_join); return; @@ -581,7 +577,9 @@ std::shared_ptr tryDirectJoin(const std::shared_ptr tryDirectJoin(const std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_jo if (!table_join->oneDisjunct() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Only `hash` join supports multiple ORs for keys in JOIN ON section"); - /// Direct JOIN with special storages that support key value access. For example JOIN with Dictionary. + /// Direct JOIN with special storages that support key value access. For example JOIN with Dictionary if (table_join->isEnabledAlgorithm(JoinAlgorithm::DIRECT)) { JoinPtr direct_join = tryDirectJoin(table_join, right_table_expression, right_table_expression_header, planner_context); diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 997d7841255..d305249e789 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -31,7 +31,7 @@ namespace DB * Example: SELECT * FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON toString(t1.id) = toString(t2.id). * toString(t1.id) = toString(t2.id) is JOIN keys section. Where toString(t1.id) is left key, and toString(t2.id) is right key. * - * During query planning JOIN ON section must be represented using join clause structure. It is important to split + * During query planning JOIN ON section represented using join clause structure. It is important to split * keys and conditions. And for each action detect from which stream it can be performed. * * We have 2 streams, left stream and right stream. @@ -79,26 +79,16 @@ public: return left_key_nodes; } - /// Get right key nodes - const ActionsDAG::NodeRawConstPtrs & getRightKeyNodes() const - { - return right_key_nodes; - } - /// Get left key nodes ActionsDAG::NodeRawConstPtrs & getLeftKeyNodes() { return left_key_nodes; } - bool hasASOF() const + /// Get right key nodes + const ActionsDAG::NodeRawConstPtrs & getRightKeyNodes() const { - return !asof_conditions.empty(); - } - - const std::vector & getASOFConditions() const - { - return asof_conditions; + return right_key_nodes; } /// Get right key nodes @@ -107,23 +97,36 @@ public: return right_key_nodes; } + /// Returns true if JOIN clause has ASOF conditions, false otherwise + bool hasASOF() const + { + return !asof_conditions.empty(); + } + + /// Get ASOF conditions + const std::vector & getASOFConditions() const + { + return asof_conditions; + } + /// Get left filter condition nodes const ActionsDAG::NodeRawConstPtrs & getLeftFilterConditionNodes() const { return left_filter_condition_nodes; } + /// Get left filter condition nodes + ActionsDAG::NodeRawConstPtrs & getLeftFilterConditionNodes() + { + return left_filter_condition_nodes; + } + /// Get right filter condition nodes const ActionsDAG::NodeRawConstPtrs & getRightFilterConditionNodes() const { return right_filter_condition_nodes; } - ActionsDAG::NodeRawConstPtrs & getLeftFilterConditionNodes() - { - return left_filter_condition_nodes; - } - /// Get right filter condition nodes ActionsDAG::NodeRawConstPtrs & getRightFilterConditionNodes() { @@ -183,7 +186,7 @@ std::optional tryExtractConstantFromJoinNode(const QueryTreeNodePtr & join /** Choose JOIN algorithm for table join, right table expression, right table expression header and planner context. * Table join structure can be modified during JOIN algorithm choosing for special JOIN algorithms. - * For example JOIN with Dictionary enigne, or JOIN with JOIN engine. + * For example JOIN with Dictionary engine, or JOIN with JOIN engine. */ std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_join, const QueryTreeNodePtr & right_table_expression, diff --git a/src/Planner/PlannerSorting.cpp b/src/Planner/PlannerSorting.cpp index 2ce497b7bbd..5ae8bd1e21b 100644 --- a/src/Planner/PlannerSorting.cpp +++ b/src/Planner/PlannerSorting.cpp @@ -24,7 +24,7 @@ namespace std::pair extractWithFillValue(const QueryTreeNodePtr & node) { - auto constant_value = node->getConstantValue(); + const auto & constant_value = node->getConstantValue(); std::pair result; result.first = constant_value.getValue(); @@ -38,7 +38,7 @@ std::pair extractWithFillValue(const QueryTreeNodePtr & node std::pair> extractWithFillStepValue(const QueryTreeNodePtr & node) { - auto constant_value = node->getConstantValue(); + const auto & constant_value = node->getConstantValue(); const auto & constant_node_result_type = constant_value.getType(); if (const auto * type_interval = typeid_cast(constant_node_result_type.get())) @@ -76,7 +76,8 @@ FillColumnDescription extractWithFillDescription(const SortNode & sort_node) } else { - fill_column_description.fill_step = Field(sort_node.getSortDirection() == SortDirection::ASCENDING ? 1 : -1); + auto direction_value = sort_node.getSortDirection() == SortDirection::ASCENDING ? static_cast(1) : static_cast(-1); + fill_column_description.fill_step = Field(direction_value); } if (applyVisitor(FieldVisitorAccurateEquals(), fill_column_description.fill_step, Field{0})) diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index c977c9a2090..4fe60a18099 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -85,8 +85,10 @@ std::vector extractWindowDescriptions(const QueryTreeNodes & } const auto & arguments_nodes = window_function_node_typed.getArguments().getNodes(); - window_function.argument_names.reserve(arguments_nodes.size()); - window_function.argument_types.reserve(arguments_nodes.size()); + size_t arguments_nodes_size = arguments_nodes.size(); + + window_function.argument_names.reserve(arguments_nodes_size); + window_function.argument_types.reserve(arguments_nodes_size); for (const auto & argument_node : arguments_nodes) { diff --git a/src/Planner/PlannerWindowFunctions.h b/src/Planner/PlannerWindowFunctions.h index b31a5c1eb05..1552ef5a71f 100644 --- a/src/Planner/PlannerWindowFunctions.h +++ b/src/Planner/PlannerWindowFunctions.h @@ -9,10 +9,10 @@ namespace DB { -/// Extract and sort window description from query. +/// Extract window descriptions from window function nodes std::vector extractWindowDescriptions(const QueryTreeNodes & window_function_nodes, const PlannerContext & planner_context); -/** Try to sort window description in such an order that the window with the longest +/** Try to sort window descriptions in such an order that the window with the longest * sort description goes first, and all window that use its prefixes follow. */ void sortWindowDescriptions(std::vector & window_descriptions); diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h index 5db31a19475..0918c35a8ef 100644 --- a/src/Planner/TableExpressionData.h +++ b/src/Planner/TableExpressionData.h @@ -22,7 +22,7 @@ class TableExpressionData public: using ColumnNameToColumnIdentifier = std::unordered_map; - using ColumnIdentifierToColumnName = std::unordered_map; + using ColumnIdentifierToColumnName = std::unordered_map; /// Return true if column with name exists, false otherwise bool hasColumn(const std::string & column_name) const @@ -32,6 +32,8 @@ public: /** Add column in table expression data. * Column identifier must be created using global planner context. + * + * Logical error exception is thrown if column already exists. */ void addColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) { @@ -64,13 +66,13 @@ public: alias_columns_names.insert(column_name); } - /// Get alias column names + /// Get alias columns names const NameSet & getAliasColumnsNames() const { return alias_columns_names; } - /// Get column names + /// Get columns names const NameSet & getColumnsNames() const { return columns_names; @@ -146,7 +148,7 @@ public: return &it->second; } - /** Cache value of storage is remote method call. + /** Returns true if storage is remote, false otherwise. * * Valid only for table and table function node. */ @@ -155,29 +157,29 @@ public: return is_remote; } - /// Set is remote value + /// Set is storage remote value void setIsRemote(bool is_remote_value) { is_remote = is_remote_value; } private: - /// Valid for table, table function, query table expression nodes + /// Valid for table, table function, query, union table expression nodes NamesAndTypesList columns; - /// Valid for table, table function, query table expression nodes + /// Valid for table, table function, query, union table expression nodes NameSet columns_names; /// Valid only for table table expression node NameSet alias_columns_names; - /// Valid for table, table function, query table expression nodes + /// Valid for table, table function, query, union table expression nodes ColumnNameToColumnIdentifier column_name_to_column_identifier; - /// Valid for table, table function, query table expression nodes + /// Valid for table, table function, query, union table expression nodes ColumnIdentifierToColumnName column_identifier_to_column_name; - /// Cached value if table expression receives data from remote server + /// Is storage remote bool is_remote = false; }; diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 9b18a3a0ab5..74918285453 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -32,6 +32,7 @@ String dumpQueryPlan(QueryPlan & query_plan) { WriteBufferFromOwnString query_plan_buffer; query_plan.explainPlan(query_plan_buffer, QueryPlan::ExplainPlanOptions{true, true, true, true}); + return query_plan_buffer.str(); } @@ -40,6 +41,7 @@ String dumpQueryPipeline(QueryPlan & query_plan) QueryPlan::ExplainPipelineOptions explain_pipeline; WriteBufferFromOwnString query_pipeline_buffer; query_plan.explainPipeline(query_pipeline_buffer, explain_pipeline); + return query_pipeline_buffer.str(); } @@ -47,26 +49,26 @@ Block buildCommonHeaderForUnion(const Blocks & queries_headers) { size_t num_selects = queries_headers.size(); Block common_header = queries_headers.front(); - size_t num_columns = common_header.columns(); + size_t columns_size = common_header.columns(); - for (size_t query_num = 1; query_num < num_selects; ++query_num) + for (size_t query_number = 1; query_number < num_selects; ++query_number) { - if (queries_headers.at(query_num).columns() != num_columns) + if (queries_headers.at(query_number).columns() != columns_size) throw Exception(ErrorCodes::TYPE_MISMATCH, "Different number of columns in UNION elements: {} and {}", common_header.dumpNames(), - queries_headers[query_num].dumpNames()); + queries_headers[query_number].dumpNames()); } std::vector columns(num_selects); - for (size_t column_num = 0; column_num < num_columns; ++column_num) + for (size_t column_number = 0; column_number < columns_size; ++column_number) { for (size_t i = 0; i < num_selects; ++i) - columns[i] = &queries_headers[i].getByPosition(column_num); + columns[i] = &queries_headers[i].getByPosition(column_number); - ColumnWithTypeAndName & result_elem = common_header.getByPosition(column_num); - result_elem = getLeastSuperColumn(columns); + ColumnWithTypeAndName & result_element = common_header.getByPosition(column_number); + result_element = getLeastSuperColumn(columns); } return common_header; @@ -175,20 +177,18 @@ ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & express ActionsDAGPtr action_dag = std::make_shared(input_columns); PlannerActionsVisitor actions_visitor(planner_context); auto expression_dag_index_nodes = actions_visitor.visit(action_dag, expression_node); - action_dag->getOutputs().clear(); - - for (auto & expression_dag_index_node : expression_dag_index_nodes) - action_dag->getOutputs().push_back(expression_dag_index_node); + action_dag->getOutputs() = std::move(expression_dag_index_nodes); return action_dag; } bool sortDescriptionIsPrefix(const SortDescription & prefix, const SortDescription & full) { - if (prefix.size() > full.size()) + size_t prefix_size = prefix.size(); + if (prefix_size > full.size()) return false; - for (size_t i = 0; i < prefix.size(); ++i) + for (size_t i = 0; i < prefix_size; ++i) { if (full[i] != prefix[i]) return false; @@ -204,7 +204,7 @@ bool queryHasArrayJoinInJoinTree(const QueryTreeNodePtr & query_node) std::vector join_tree_nodes_to_process; join_tree_nodes_to_process.push_back(query_node_typed.getJoinTree()); - while (join_tree_nodes_to_process.empty()) + while (!join_tree_nodes_to_process.empty()) { auto join_tree_node_to_process = join_tree_nodes_to_process.back(); join_tree_nodes_to_process.pop_back(); @@ -253,7 +253,7 @@ bool queryHasWithTotalsInAnySubqueryInJoinTree(const QueryTreeNodePtr & query_no std::vector join_tree_nodes_to_process; join_tree_nodes_to_process.push_back(query_node_typed.getJoinTree()); - while (join_tree_nodes_to_process.empty()) + while (!join_tree_nodes_to_process.empty()) { auto join_tree_node_to_process = join_tree_nodes_to_process.back(); join_tree_nodes_to_process.pop_back(); diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 82026b1d72e..909cea3bf8f 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -48,7 +48,7 @@ ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & express /// Returns true if prefix sort description is prefix of full sort descriptor, false otherwise bool sortDescriptionIsPrefix(const SortDescription & prefix, const SortDescription & full); -/// Returns true if query node JOIN TREE contains ARRAY JOIN node +/// Returns true if query node JOIN TREE contains ARRAY JOIN node, false otherwise bool queryHasArrayJoinInJoinTree(const QueryTreeNodePtr & query_node); /** Returns true if query node JOIN TREE contains QUERY node with WITH TOTALS, false otherwise. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 322790b6b14..b81fafbc578 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -490,12 +490,8 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( { const auto & table_expression_modifiers = *select_query_info.table_expression_modifiers; final = table_expression_modifiers.hasFinal(); - - if (table_expression_modifiers.hasSampleSizeRatio()) - sample_size_ratio = table_expression_modifiers.getSampleSizeRatio(); - - if (table_expression_modifiers.hasSampleOffsetRatio()) - sample_offset_ratio = table_expression_modifiers.getSampleSizeRatio(); + sample_size_ratio = table_expression_modifiers.getSampleSizeRatio(); + sample_offset_ratio = table_expression_modifiers.getSampleOffsetRatio(); } else { From 8ac7f4423b8bdbcb5a98fa4866d553acdbd32345 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 20 Oct 2022 12:32:52 +0200 Subject: [PATCH 151/188] Fixed tests --- src/Parsers/ASTColumnsMatcher.cpp | 10 +++++----- .../0_stateless/02342_analyzer_compound_types.sql | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Parsers/ASTColumnsMatcher.cpp b/src/Parsers/ASTColumnsMatcher.cpp index 06980208774..0fc6847de68 100644 --- a/src/Parsers/ASTColumnsMatcher.cpp +++ b/src/Parsers/ASTColumnsMatcher.cpp @@ -112,11 +112,8 @@ void ASTColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatSt settings.ostr << ")"; /// Format column transformers - size_t children_size = children.size(); - - for (size_t i = 1; i < children_size; ++i) + for (const auto & child : children) { - const auto & child = children[i]; settings.ostr << ' '; child->formatImpl(settings, state, frame); } @@ -236,8 +233,11 @@ void ASTQualifiedColumnsListMatcher::formatImpl(const FormatSettings & settings, settings.ostr << ")"; /// Format column transformers - for (const auto & child : children) + size_t children_size = children.size(); + + for (size_t i = 1; i < children_size; ++i) { + const auto & child = children[i]; settings.ostr << ' '; child->formatImpl(settings, state, frame); } diff --git a/tests/queries/0_stateless/02342_analyzer_compound_types.sql b/tests/queries/0_stateless/02342_analyzer_compound_types.sql index abe43ce101d..26f57ef57d0 100644 --- a/tests/queries/0_stateless/02342_analyzer_compound_types.sql +++ b/tests/queries/0_stateless/02342_analyzer_compound_types.sql @@ -16,7 +16,7 @@ CREATE TABLE test_table ( id UInt64, value Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) -) ENGINE=TinyLog; +) ENGINE=MergeTree ORDER BY id; INSERT INTO test_table VALUES (0, (('value_0_level_1', 'value_1_level_1'), 'value_1_level_0')); @@ -99,9 +99,9 @@ CREATE TABLE test_table ( id UInt64, value Array(Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String)) -) ENGINE=TinyLog; +) ENGINE=MergeTree ORDER BY id; -INSERT INTO test_table VALUES (0, [('value_0_level_1', 'value_1_level_1')], ['value_1_level_0']); +INSERT INTO test_table VALUES (0, [('value_0_level_1', 'value_1_level_1')], array('value_1_level_0')); DESCRIBE (SELECT * FROM test_table); SELECT * FROM test_table; @@ -150,9 +150,9 @@ CREATE TABLE test_table ( id UInt64, value Nested (value_0_level_0 Nested(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) -) ENGINE=TinyLog; +) ENGINE=MergeTree ORDER BY id; -INSERT INTO test_table VALUES (0, [[('value_0_level_1', 'value_1_level_1')]], ['value_1_level_0']); +INSERT INTO test_table VALUES (0, [[('value_0_level_1', 'value_1_level_1')]], array('value_1_level_0')); DESCRIBE (SELECT * FROM test_table); SELECT * FROM test_table; From 3862dd96fea8609dd2b82be4e9a253d8cf3bd103 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 20 Oct 2022 13:45:32 +0200 Subject: [PATCH 152/188] Fixed tests --- src/Analyzer/IQueryTreeNode.cpp | 29 ++++++++++++++++++++++- src/Planner/PlannerExpressionAnalysis.cpp | 8 +++---- 2 files changed, 32 insertions(+), 5 deletions(-) diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index dbc67749a8b..ea2412eadb2 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -53,9 +53,31 @@ IQueryTreeNode::IQueryTreeNode(size_t children_size) children.resize(children_size); } +namespace +{ + +using NodePair = std::pair; + +struct NodePairHash +{ + size_t operator()(const NodePair & node_pair) const + { + auto hash = std::hash(); + + size_t result = 0; + boost::hash_combine(result, hash(node_pair.first)); + boost::hash_combine(result, hash(node_pair.second)); + + return result; + } +}; + +} + bool IQueryTreeNode::isEqual(const IQueryTreeNode & rhs) const { - std::vector> nodes_to_process; + std::vector nodes_to_process; + std::unordered_set equals_pairs; nodes_to_process.emplace_back(this, &rhs); @@ -67,6 +89,9 @@ bool IQueryTreeNode::isEqual(const IQueryTreeNode & rhs) const const auto * lhs_node_to_compare = nodes_to_compare.first; const auto * rhs_node_to_compare = nodes_to_compare.second; + if (equals_pairs.contains(std::make_pair(lhs_node_to_compare, rhs_node_to_compare))) + continue; + assert(lhs_node_to_compare); assert(rhs_node_to_compare); @@ -121,6 +146,8 @@ bool IQueryTreeNode::isEqual(const IQueryTreeNode & rhs) const nodes_to_process.emplace_back(lhs_strong_pointer.get(), rhs_strong_pointer.get()); } + + equals_pairs.emplace(lhs_node_to_compare, rhs_node_to_compare); } return true; diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 36bd777cf5a..b034edf97d8 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -208,7 +208,7 @@ std::optional analyzeAggregation(QueryTreeNodePtr & q return aggregation_analysis_result; } -/** Construct aggregation analysis result if query tree has window functions. +/** Construct window analysis result if query tree has window functions. * Actions before window functions are added into actions chain, if result is not null optional. */ std::optional analyzeWindow(QueryTreeNodePtr & query_tree, @@ -323,8 +323,8 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, if (projection_columns_size != projection_outputs_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "QueryTree projection nodes size mismatch. Expected {}. Actual {}", - projection_outputs_size, - projection_columns_size); + projection_columns_size, + projection_outputs_size); for (size_t i = 0; i < projection_outputs_size; ++i) { @@ -487,7 +487,7 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(QueryTreeNodePtr if (having_action_step_index_optional && having_analysis_result_optional) { - auto & having_analysis_result = *where_analysis_result_optional; + auto & having_analysis_result = *having_analysis_result_optional; auto & having_actions_chain_node = actions_chain.at(*having_action_step_index_optional); having_analysis_result.remove_filter_column = !having_actions_chain_node->getChildRequiredOutputColumnsNames().contains(having_analysis_result.filter_column_name); expressions_analysis_result.addHaving(std::move(having_analysis_result)); From 9a02d25c0b96a7f20b2cc142368eb407ed7a8d1e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 20 Oct 2022 17:52:52 +0200 Subject: [PATCH 153/188] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Planner/ActionsChain.cpp | 47 ++++++++++++---- src/Planner/ActionsChain.h | 23 +++----- src/Planner/Planner.cpp | 22 ++++---- src/Planner/Planner.h | 2 +- src/Planner/PlannerActionsVisitor.cpp | 17 +++--- src/Planner/PlannerActionsVisitor.h | 6 +-- src/Planner/PlannerAggregation.cpp | 54 ++++++++----------- src/Planner/PlannerContext.h | 4 +- .../02342_analyzer_compound_types.sql | 4 +- 10 files changed, 96 insertions(+), 85 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 4f9ddd48e81..de3dc434ebf 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1451,7 +1451,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, size if (query_node) query_node->performConstantFolding(std::move(constant_value)); else if (union_node) - query_node->performConstantFolding(std::move(constant_value)); + union_node->performConstantFolding(std::move(constant_value)); return; } diff --git a/src/Planner/ActionsChain.cpp b/src/Planner/ActionsChain.cpp index 21f663e1a8a..594d26a679c 100644 --- a/src/Planner/ActionsChain.cpp +++ b/src/Planner/ActionsChain.cpp @@ -11,26 +11,51 @@ namespace DB { +ActionsChainStep::ActionsChainStep(ActionsDAGPtr actions_, AvailableOutputColumnsStrategy available_output_columns_stategy_) + : actions(std::move(actions_)) + , available_output_columns_strategy(available_output_columns_stategy_) +{ + initialize(); +} + +ActionsChainStep::ActionsChainStep(ActionsDAGPtr actions_, + AvailableOutputColumnsStrategy available_output_columns_stategy_, + ColumnsWithTypeAndName additional_output_columns_) + : actions(std::move(actions_)) + , available_output_columns_strategy(available_output_columns_stategy_) + , additional_output_columns(std::move(additional_output_columns_)) +{ + initialize(); +} + + void ActionsChainStep::finalizeInputAndOutputColumns(const NameSet & child_input_columns) { child_required_output_columns_names.clear(); - std::vector required_output_nodes; auto child_input_columns_copy = child_input_columns; + std::unordered_set output_nodes_names; + output_nodes_names.reserve(actions->getOutputs().size()); + + for (auto & output_node : actions->getOutputs()) + output_nodes_names.insert(output_node->result_name); + for (const auto & node : actions->getNodes()) { auto it = child_input_columns_copy.find(node.result_name); if (it == child_input_columns_copy.end()) continue; - child_required_output_columns_names.insert(node.result_name); - required_output_nodes.push_back(&node); child_input_columns_copy.erase(it); - } + child_required_output_columns_names.insert(node.result_name); - for (auto & required_output_node : required_output_nodes) - actions->addOrReplaceInOutputs(*required_output_node); + if (output_nodes_names.contains(node.result_name)) + continue; + + actions->getOutputs().push_back(&node); + output_nodes_names.insert(node.result_name); + } actions->removeUnusedActions(); /// TODO: Analyzer fix ActionsDAG input and constant nodes with same name @@ -49,6 +74,7 @@ void ActionsChainStep::dump(WriteBuffer & buffer) const for (const auto & column : additional_output_columns) buffer << "Name " << column.name << " type " << column.type->getName() << '\n'; } + if (!child_required_output_columns_names.empty()) { buffer << "Child required output columns " << boost::join(child_required_output_columns_names, ", "); @@ -121,13 +147,13 @@ void ActionsChain::finalize() void ActionsChain::dump(WriteBuffer & buffer) const { - size_t nodes_size = steps.size(); + size_t steps_size = steps.size(); - for (size_t i = 0; i < nodes_size; ++i) + for (size_t i = 0; i < steps_size; ++i) { - const auto & node = steps[i]; + const auto & step = steps[i]; buffer << "Step " << i << '\n'; - node->dump(buffer); + step->dump(buffer); buffer << '\n'; } @@ -137,6 +163,7 @@ String ActionsChain::dump() const { WriteBufferFromOwnString buffer; dump(buffer); + return buffer.str(); } diff --git a/src/Planner/ActionsChain.h b/src/Planner/ActionsChain.h index 89f8ad2a319..e2791ab7e35 100644 --- a/src/Planner/ActionsChain.h +++ b/src/Planner/ActionsChain.h @@ -19,9 +19,9 @@ namespace ErrorCodes * We want to reuse expr(id) from previous expressions step, and not recalculate it in projection. * To do this we build a chain of all query action steps. * For example: - * 1. before where - * 2. before order by - * 3. projection + * 1. Before where. + * 2. Before order by. + * 3. Projection. * * Initially root of chain is initialized with join tree query plan header. * Each next chain step, must be initialized with previous step available output columns. @@ -56,22 +56,11 @@ public: * If available output columns strategy is ALL_NODES, then available output columns initialized using actions dag nodes. * If available output columns strategy is OUTPUT_NODES, then available output columns initialized using actions dag output nodes. */ - explicit ActionsChainStep(ActionsDAGPtr actions_, AvailableOutputColumnsStrategy available_output_columns_stategy_ = AvailableOutputColumnsStrategy::ALL_NODES) - : actions(std::move(actions_)) - , available_output_columns_strategy(available_output_columns_stategy_) - { - initialize(); - } + explicit ActionsChainStep(ActionsDAGPtr actions_, AvailableOutputColumnsStrategy available_output_columns_stategy_ = AvailableOutputColumnsStrategy::ALL_NODES); explicit ActionsChainStep(ActionsDAGPtr actions_, AvailableOutputColumnsStrategy available_output_columns_stategy_, - ColumnsWithTypeAndName additional_output_columns_) - : actions(std::move(actions_)) - , available_output_columns_strategy(available_output_columns_stategy_) - , additional_output_columns(std::move(additional_output_columns_)) - { - initialize(); - } + ColumnsWithTypeAndName additional_output_columns_); /// Get actions ActionsDAGPtr & getActions() @@ -225,7 +214,7 @@ public: return steps.back()->getAvailableOutputColumns(); } - /// Get last step available output columns or throw exception if chain is empty + /// Get last step available output columns or null if chain is empty const ColumnsWithTypeAndName * getLastStepAvailableOutputColumnsOrNull() const { if (steps.empty()) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index ca991b76c94..2e128c93d08 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -136,7 +136,7 @@ void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, const SelectQ PreparedSets::SubqueriesForSets subqueries_for_sets; const auto & set_key_to_planner_set = planner_context->getRegisteredSets(); - for (auto [key, planner_set] : set_key_to_planner_set) + for (const auto & [key, planner_set] : set_key_to_planner_set) { const auto subquery_node = planner_set.getSubqueryNode(); if (!subquery_node) @@ -188,19 +188,18 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_, ContextPtr context_) : query_tree(query_tree_) , select_query_options(select_query_options_) - , planner_context(std::make_shared(context_, std::make_shared())) + , planner_context(std::make_shared(std::move(context_), std::make_shared())) { initialize(); } -/// Initialize interpreter with query tree after query analysis phase and global planner context Planner::Planner(const QueryTreeNodePtr & query_tree_, const SelectQueryOptions & select_query_options_, ContextPtr context_, GlobalPlannerContextPtr global_planner_context_) : query_tree(query_tree_) , select_query_options(select_query_options_) - , planner_context(std::make_shared(context_, std::move(global_planner_context_))) + , planner_context(std::make_shared(std::move(context_), std::move(global_planner_context_))) { initialize(); } @@ -244,7 +243,7 @@ void Planner::initialize() if (need_apply_query_settings) updated_context->applySettingsChanges(query_node->getSettingsChanges()); - /// Disable two-level aggregation due to version incompatibility. + /// Disable two-level aggregation due to version incompatibility if (need_to_disable_two_level_aggregation) { updated_context->setSetting("group_by_two_level_threshold", Field(0)); @@ -514,6 +513,7 @@ void Planner::buildQueryPlanIfNeeded() { const auto & having_analysis_result = expression_analysis_result.getHaving(); bool final = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube(); + having_executed = true; auto totals_having_step = std::make_unique( query_plan.getCurrentDataStream(), @@ -575,7 +575,7 @@ void Planner::buildQueryPlanIfNeeded() query_plan.getCurrentDataStream(), window_description.full_sort_description, settings.max_block_size, - 0 /* LIMIT */, + 0 /*limit*/, SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), settings.max_bytes_before_remerge_sort, settings.remerge_sort_lowered_memory_bytes_ratio, @@ -664,16 +664,16 @@ void Planner::buildQueryPlanIfNeeded() UInt64 partial_sorting_limit = 0; - /// Partial sort can be done if there is LIMIT, but no DISTINCT, LIMIT WITH TIES, LIMIT BY, ARRAY JOIN. + /// Partial sort can be done if there is LIMIT, but no DISTINCT, LIMIT WITH TIES, LIMIT BY, ARRAY JOIN if (limit_length != 0 && !query_node.isDistinct() && !query_node.hasLimitBy() && !query_node.isLimitWithTies() && !query_has_array_join_in_join_tree && limit_length <= std::numeric_limits::max() - limit_offset) { partial_sorting_limit = limit_length + limit_offset; } - const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); + const Settings & settings = query_context->getSettingsRef(); - /// Merge the sorted blocks. + /// Merge the sorted blocks auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), sort_description, @@ -683,7 +683,7 @@ void Planner::buildQueryPlanIfNeeded() settings.max_bytes_before_remerge_sort, settings.remerge_sort_lowered_memory_bytes_ratio, settings.max_bytes_before_external_sort, - planner_context->getQueryContext()->getTempDataOnDisk(), + query_context->getTempDataOnDisk(), settings.min_free_disk_space_for_temporary_data, settings.optimize_sorting_by_input_stream_properties); @@ -814,7 +814,7 @@ void Planner::buildQueryPlanIfNeeded() if (query_node.hasLimit()) { - const Settings & settings = planner_context->getQueryContext()->getSettingsRef(); + const Settings & settings = query_context->getSettingsRef(); bool always_read_till_end = settings.exact_rows_before_limit; bool limit_with_ties = query_node.isLimitWithTies(); diff --git a/src/Planner/Planner.h b/src/Planner/Planner.h index 389df4365fa..03f8e19df56 100644 --- a/src/Planner/Planner.h +++ b/src/Planner/Planner.h @@ -24,7 +24,7 @@ public: const SelectQueryOptions & select_query_options_, ContextPtr context_); - /// Initialize interpreter with query tree after query analysis phase and global planner context + /// Initialize planner with query tree after query analysis phase and global planner context Planner(const QueryTreeNodePtr & query_tree_, const SelectQueryOptions & select_query_options_, ContextPtr context_, diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 10484eef454..d47ecb238fc 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -223,7 +223,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi return visitQueryOrUnion(node); throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Expected only column, constant or function node. Actual {}", + "Expected column, constant, function, query or union node. Actual {}", node->formatASTForErrorMessage()); } @@ -285,13 +285,16 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi "Lambda {} is not resolved during query analysis", lambda_node.formatASTForErrorMessage()); + auto & lambda_arguments_nodes = lambda_node.getArguments().getNodes(); + size_t lambda_arguments_nodes_size = lambda_arguments_nodes.size(); + NamesAndTypesList lambda_arguments_names_and_types; - for (auto & lambda_node_argument : lambda_node.getArguments().getNodes()) + for (size_t i = 0; i < lambda_arguments_nodes_size; ++i) { - auto lambda_argument_name = lambda_node_argument->getName(); - auto lambda_argument_type = lambda_node_argument->getResultType(); - lambda_arguments_names_and_types.emplace_back(lambda_argument_name, lambda_argument_type); + const auto & lambda_argument_name = lambda_node.getArgumentNames().at(i); + auto lambda_argument_type = lambda_arguments_nodes[i]->getResultType(); + lambda_arguments_names_and_types.emplace_back(lambda_argument_name, std::move(lambda_argument_type)); } size_t previous_scope_node_actions_stack_index = actions_stack.size() - 1; @@ -332,7 +335,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi if (level == actions_stack.size()) --level; - actions_stack[level].addFunctionIfNecessary(lambda_node_name, lambda_children, function_capture); + actions_stack[level].addFunctionIfNecessary(lambda_node_name, std::move(lambda_children), std::move(function_capture)); size_t actions_stack_size = actions_stack.size(); for (size_t i = level + 1; i < actions_stack_size; ++i) @@ -350,7 +353,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma auto in_second_argument = function_node.getArguments().getNodes().at(1); auto set_key = planner_context->createSetKey(in_second_argument); - auto planner_set = planner_context->getSetOrThrow(set_key); + const auto & planner_set = planner_context->getSetOrThrow(set_key); ColumnWithTypeAndName column; column.name = set_key; diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 8cc75461d6c..405031daa40 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -24,11 +24,11 @@ using PlannerContextPtr = std::shared_ptr; * 1. Table expression data for table expression nodes is collected in planner context. * For column node, that has column table expression source, identifier for column name in table expression data * is used as action dag node name. - * 2. Sets for IN functions are already collected in planner global context. + * 2. Sets for IN functions are already collected in planner context. * * During actions build, there is special handling for following functions: * 1. Aggregate functions are added in actions dag as INPUT nodes. Aggregate functions arguments are not added. - * 2. For function `in` and its variants, already collected sets from global context are used. + * 2. For function `in` and its variants, already collected sets from planner context are used. */ class PlannerActionsVisitor { @@ -45,7 +45,7 @@ private: const PlannerContextPtr planner_context; }; -/** Calculate query tree expression node name action dag name and add them into node to name map. +/** Calculate query tree expression node action dag name and add them into node to name map. * If node exists in map, name from map is used. * * For column node column node identifier from planner context is used. diff --git a/src/Planner/PlannerAggregation.cpp b/src/Planner/PlannerAggregation.cpp index aabf186b49c..3322ef9364f 100644 --- a/src/Planner/PlannerAggregation.cpp +++ b/src/Planner/PlannerAggregation.cpp @@ -81,7 +81,7 @@ public: if (it == aggegation_key_to_index.end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument of GROUPING function {} is not a part of GROUP BY clause", - argument->formatASTForErrorMessage()); + argument->formatASTForErrorMessage()); arguments_indexes.push_back(it->second); } @@ -98,30 +98,30 @@ public: { auto grouping_ordinary_function = std::make_shared(arguments_indexes, force_grouping_standard_compatibility); auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_ordinary_function)); - function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); + function_node->resolveAsFunction(std::move(grouping_ordinary_function_adaptor), std::make_shared()); break; } case GroupByKind::ROLLUP: { - auto grouping_ordinary_function = std::make_shared(arguments_indexes, aggregation_keys_size, force_grouping_standard_compatibility); - auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_ordinary_function)); - function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); + auto grouping_rollup_function = std::make_shared(arguments_indexes, aggregation_keys_size, force_grouping_standard_compatibility); + auto grouping_rollup_function_adaptor = std::make_shared(std::move(grouping_rollup_function)); + function_node->resolveAsFunction(std::move(grouping_rollup_function_adaptor), std::make_shared()); function_node->getArguments().getNodes().push_back(std::move(grouping_set_argument_column)); break; } case GroupByKind::CUBE: { - auto grouping_ordinary_function = std::make_shared(arguments_indexes, aggregation_keys_size, force_grouping_standard_compatibility); - auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_ordinary_function)); - function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); + auto grouping_cube_function = std::make_shared(arguments_indexes, aggregation_keys_size, force_grouping_standard_compatibility); + auto grouping_cube_function_adaptor = std::make_shared(std::move(grouping_cube_function)); + function_node->resolveAsFunction(std::move(grouping_cube_function_adaptor), std::make_shared()); function_node->getArguments().getNodes().push_back(std::move(grouping_set_argument_column)); break; } case GroupByKind::GROUPING_SETS: { auto grouping_grouping_sets_function = std::make_shared(arguments_indexes, grouping_sets_keys_indices, force_grouping_standard_compatibility); - auto grouping_ordinary_function_adaptor = std::make_shared(std::move(grouping_grouping_sets_function)); - function_node->resolveAsFunction(grouping_ordinary_function_adaptor, std::make_shared()); + auto grouping_grouping_sets_function_adaptor = std::make_shared(std::move(grouping_grouping_sets_function)); + function_node->resolveAsFunction(std::move(grouping_grouping_sets_function_adaptor), std::make_shared()); function_node->getArguments().getNodes().push_back(std::move(grouping_set_argument_column)); break; } @@ -147,8 +147,17 @@ void resolveGroupingFunctions(QueryTreeNodePtr & node, const GroupingSetsParamsList & grouping_sets_parameters_list, const PlannerContext & planner_context) { + auto & query_node_typed = node->as(); + GroupingFunctionResolveVisitor visitor(group_by_kind, aggregation_keys, grouping_sets_parameters_list, planner_context); - visitor.visit(node); + + if (query_node_typed.hasHaving()) + visitor.visit(query_node_typed.getHaving()); + + if (query_node_typed.hasOrderBy()) + visitor.visit(query_node_typed.getOrderByNode()); + + visitor.visit(query_node_typed.getProjectionNode()); } } @@ -168,26 +177,7 @@ void resolveGroupingFunctions(QueryTreeNodePtr & query_node, else if (query_node_typed.isGroupByWithGroupingSets()) group_by_kind = GroupByKind::GROUPING_SETS; - if (query_node_typed.hasHaving()) - { - resolveGroupingFunctions(query_node_typed.getHaving(), - group_by_kind, - aggregation_keys, - grouping_sets_parameters_list, - planner_context); - } - - resolveGroupingFunctions(query_node_typed.getOrderByNode(), - group_by_kind, - aggregation_keys, - grouping_sets_parameters_list, - planner_context); - - resolveGroupingFunctions(query_node_typed.getProjectionNode(), - group_by_kind, - aggregation_keys, - grouping_sets_parameters_list, - planner_context); + resolveGroupingFunctions(query_node, group_by_kind, aggregation_keys, grouping_sets_parameters_list, planner_context); } AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodes & aggregate_function_nodes, const PlannerContext & planner_context) @@ -225,7 +215,7 @@ AggregateDescriptions extractAggregateDescriptions(const QueryTreeNodes & aggreg aggregate_description.argument_names.emplace_back(std::move(argument_node_name)); } - aggregate_description.column_name = node_name; + aggregate_description.column_name = std::move(node_name); aggregate_descriptions.push_back(std::move(aggregate_description)); } diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index b4f530cae65..63874bf7ab9 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -66,11 +66,13 @@ public: , subquery_node(std::move(subquery_node_)) {} + /// Get set const SetPtr & getSet() const { return set; } + /// Get subquery node const QueryTreeNodePtr & getSubqueryNode() const { return subquery_node; @@ -135,7 +137,7 @@ public: */ TableExpressionData * getTableExpressionDataOrNull(const QueryTreeNodePtr & table_expression_node); - /// Get table expression node to data map read only map + /// Get table expression node to data read only map const std::unordered_map & getTableExpressionNodeToData() const { return table_expression_node_to_data; diff --git a/tests/queries/0_stateless/02342_analyzer_compound_types.sql b/tests/queries/0_stateless/02342_analyzer_compound_types.sql index 26f57ef57d0..6752064d13f 100644 --- a/tests/queries/0_stateless/02342_analyzer_compound_types.sql +++ b/tests/queries/0_stateless/02342_analyzer_compound_types.sql @@ -101,7 +101,7 @@ CREATE TABLE test_table value Array(Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String)) ) ENGINE=MergeTree ORDER BY id; -INSERT INTO test_table VALUES (0, [('value_0_level_1', 'value_1_level_1')], array('value_1_level_0')); +INSERT INTO test_table VALUES (0, [('value_0_level_1', 'value_1_level_1')], ['value_1_level_0']); DESCRIBE (SELECT * FROM test_table); SELECT * FROM test_table; @@ -152,7 +152,7 @@ CREATE TABLE test_table value Nested (value_0_level_0 Nested(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String) ) ENGINE=MergeTree ORDER BY id; -INSERT INTO test_table VALUES (0, [[('value_0_level_1', 'value_1_level_1')]], array('value_1_level_0')); +INSERT INTO test_table VALUES (0, [[('value_0_level_1', 'value_1_level_1')]], ['value_1_level_0']); DESCRIBE (SELECT * FROM test_table); SELECT * FROM test_table; From 5ebdf847ff444a59fa653d6e1b1410aad31baf95 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 11:27:31 +0300 Subject: [PATCH 154/188] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b02d3d9ac84..f2b8e9004b0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -37,7 +37,7 @@ #### Performance Improvement * Relax the "Too many parts" threshold. This closes [#6551](https://github.com/ClickHouse/ClickHouse/issues/6551). Now ClickHouse will allow more parts in a partition if the average part size is large enough (at least 10 GiB). This allows to have up to petabytes of data in a single partition of a single table on a single server, which is possible using disk shelves or object storage. [#42002](https://github.com/ClickHouse/ClickHouse/pull/42002) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Implement operator precedence element parser to resolve stack overflow issues and make the required stack size smaller. [#34892](https://github.com/ClickHouse/ClickHouse/pull/34892) ([Nikolay Degterinsky](https://github.com/evillique)). +* Implement operator precedence element parser to make the required stack size smaller. [#34892](https://github.com/ClickHouse/ClickHouse/pull/34892) ([Nikolay Degterinsky](https://github.com/evillique)). * DISTINCT in order optimization leverage sorting properties of data streams. This improvement will enable reading in order for DISTINCT if applicable (before it was necessary to provide ORDER BY for columns in DISTINCT). [#41014](https://github.com/ClickHouse/ClickHouse/pull/41014) ([Igor Nikonov](https://github.com/devcrafter)). * ColumnVector: optimize UInt8 index with AVX512VBMI. [#41247](https://github.com/ClickHouse/ClickHouse/pull/41247) ([Guo Wangyang](https://github.com/guowangy)). * Optimize the lock contentions for `ThreadGroupStatus::mutex`. The performance experiments of **SSB** (Star Schema Benchmark) on the ICX device (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads) shows that this change could bring a **2.95x** improvement of the geomean of all subcases' QPS. [#41675](https://github.com/ClickHouse/ClickHouse/pull/41675) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). From 0762783fbdd00d8b295f11631df23848e0771d5a Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Mon, 24 Oct 2022 11:58:37 +0200 Subject: [PATCH 155/188] Fix incorrect trace log line on dict reload --- src/Interpreters/ExternalLoader.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 704dff325b7..ea2b9045120 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -714,7 +714,10 @@ public: /// Object was never loaded successfully and should be reloaded. startLoading(info); } - LOG_TRACE(log, "Object '{}' is neither loaded nor failed, so it will not be reloaded as outdated.", info.name); + else + { + LOG_TRACE(log, "Object '{}' is neither loaded nor failed, so it will not be reloaded as outdated.", info.name); + } } } } From 42acdb6edc754f7eaa4ca32aae1dc922917244fa Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Mon, 24 Oct 2022 11:03:44 +0000 Subject: [PATCH 156/188] Fix flaky 02458_datediff_date32 test --- .../02458_datediff_date32.reference | 66 +++++++++---------- .../0_stateless/02458_datediff_date32.sql | 66 +++++++++---------- 2 files changed, 66 insertions(+), 66 deletions(-) diff --git a/tests/queries/0_stateless/02458_datediff_date32.reference b/tests/queries/0_stateless/02458_datediff_date32.reference index 67bfa895199..9e93af03896 100644 --- a/tests/queries/0_stateless/02458_datediff_date32.reference +++ b/tests/queries/0_stateless/02458_datediff_date32.reference @@ -1,56 +1,56 @@ -- { echo } -- Date32 vs Date32 -SELECT dateDiff('second', toDate32('1900-01-01'), toDate32('1900-01-02')); +SELECT dateDiff('second', toDate32('1927-01-01'), toDate32('1927-01-02')); 86400 -SELECT dateDiff('minute', toDate32('1900-01-01'), toDate32('1900-01-02')); +SELECT dateDiff('minute', toDate32('1927-01-01'), toDate32('1927-01-02')); 1440 -SELECT dateDiff('hour', toDate32('1900-01-01'), toDate32('1900-01-02')); +SELECT dateDiff('hour', toDate32('1927-01-01'), toDate32('1927-01-02')); 24 -SELECT dateDiff('day', toDate32('1900-01-01'), toDate32('1900-01-02')); +SELECT dateDiff('day', toDate32('1927-01-01'), toDate32('1927-01-02')); 1 -SELECT dateDiff('week', toDate32('1900-01-01'), toDate32('1900-01-08')); +SELECT dateDiff('week', toDate32('1927-01-01'), toDate32('1927-01-08')); 1 -SELECT dateDiff('month', toDate32('1900-01-01'), toDate32('1900-02-01')); +SELECT dateDiff('month', toDate32('1927-01-01'), toDate32('1927-02-01')); 1 -SELECT dateDiff('quarter', toDate32('1900-01-01'), toDate32('1900-04-01')); +SELECT dateDiff('quarter', toDate32('1927-01-01'), toDate32('1927-04-01')); 1 -SELECT dateDiff('year', toDate32('1900-01-01'), toDate32('1901-01-01')); +SELECT dateDiff('year', toDate32('1927-01-01'), toDate32('1928-01-01')); 1 -- With DateTime64 -- Date32 vs DateTime64 -SELECT dateDiff('second', toDate32('1900-01-01'), toDateTime64('1900-01-02 00:00:00', 3)); +SELECT dateDiff('second', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); 86400 -SELECT dateDiff('minute', toDate32('1900-01-01'), toDateTime64('1900-01-02 00:00:00', 3)); +SELECT dateDiff('minute', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); 1440 -SELECT dateDiff('hour', toDate32('1900-01-01'), toDateTime64('1900-01-02 00:00:00', 3)); +SELECT dateDiff('hour', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); 24 -SELECT dateDiff('day', toDate32('1900-01-01'), toDateTime64('1900-01-02 00:00:00', 3)); +SELECT dateDiff('day', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); 1 -SELECT dateDiff('week', toDate32('1900-01-01'), toDateTime64('1900-01-08 00:00:00', 3)); +SELECT dateDiff('week', toDate32('1927-01-01'), toDateTime64('1927-01-08 00:00:00', 3)); 1 -SELECT dateDiff('month', toDate32('1900-01-01'), toDateTime64('1900-02-01 00:00:00', 3)); +SELECT dateDiff('month', toDate32('1927-01-01'), toDateTime64('1927-02-01 00:00:00', 3)); 1 -SELECT dateDiff('quarter', toDate32('1900-01-01'), toDateTime64('1900-04-01 00:00:00', 3)); +SELECT dateDiff('quarter', toDate32('1927-01-01'), toDateTime64('1927-04-01 00:00:00', 3)); 1 -SELECT dateDiff('year', toDate32('1900-01-01'), toDateTime64('1901-01-01 00:00:00', 3)); +SELECT dateDiff('year', toDate32('1927-01-01'), toDateTime64('1928-01-01 00:00:00', 3)); 1 -- DateTime64 vs Date32 -SELECT dateDiff('second', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-01-02')); +SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); 86400 -SELECT dateDiff('minute', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-01-02')); +SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); 1440 -SELECT dateDiff('hour', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-01-02')); +SELECT dateDiff('hour', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); 24 -SELECT dateDiff('day', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-01-02')); +SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); 1 -SELECT dateDiff('week', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-01-08')); +SELECT dateDiff('week', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-08')); 1 -SELECT dateDiff('month', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-02-01')); +SELECT dateDiff('month', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-02-01')); 1 -SELECT dateDiff('quarter', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-04-01')); +SELECT dateDiff('quarter', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-04-01')); 1 -SELECT dateDiff('year', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1901-01-01')); +SELECT dateDiff('year', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1928-01-01')); 1 -- With DateTime -- Date32 vs DateTime @@ -123,11 +123,11 @@ SELECT dateDiff('quarter', toDate('2015-08-18'), toDate32('2015-11-18')); SELECT dateDiff('year', toDate('2015-08-18'), toDate32('2016-08-18')); 1 -- Const vs non-const columns -SELECT dateDiff('day', toDate32('1900-01-01'), materialize(toDate32('1900-01-02'))); +SELECT dateDiff('day', toDate32('1927-01-01'), materialize(toDate32('1927-01-02'))); 1 -SELECT dateDiff('day', toDate32('1900-01-01'), materialize(toDateTime64('1900-01-02 00:00:00', 3))); +SELECT dateDiff('day', toDate32('1927-01-01'), materialize(toDateTime64('1927-01-02 00:00:00', 3))); 1 -SELECT dateDiff('day', toDateTime64('1900-01-01 00:00:00', 3), materialize(toDate32('1900-01-02'))); +SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 3), materialize(toDate32('1927-01-02'))); 1 SELECT dateDiff('day', toDate32('2015-08-18'), materialize(toDateTime('2015-08-19 00:00:00'))); 1 @@ -138,11 +138,11 @@ SELECT dateDiff('day', toDate32('2015-08-18'), materialize(toDate('2015-08-19')) SELECT dateDiff('day', toDate('2015-08-18'), materialize(toDate32('2015-08-19'))); 1 -- Non-const vs const columns -SELECT dateDiff('day', materialize(toDate32('1900-01-01')), toDate32('1900-01-02')); +SELECT dateDiff('day', materialize(toDate32('1927-01-01')), toDate32('1927-01-02')); 1 -SELECT dateDiff('day', materialize(toDate32('1900-01-01')), toDateTime64('1900-01-02 00:00:00', 3)); +SELECT dateDiff('day', materialize(toDate32('1927-01-01')), toDateTime64('1927-01-02 00:00:00', 3)); 1 -SELECT dateDiff('day', materialize(toDateTime64('1900-01-01 00:00:00', 3)), toDate32('1900-01-02')); +SELECT dateDiff('day', materialize(toDateTime64('1927-01-01 00:00:00', 3)), toDate32('1927-01-02')); 1 SELECT dateDiff('day', materialize(toDate32('2015-08-18')), toDateTime('2015-08-19 00:00:00')); 1 @@ -153,11 +153,11 @@ SELECT dateDiff('day', materialize(toDate32('2015-08-18')), toDate('2015-08-19') SELECT dateDiff('day', materialize(toDate('2015-08-18')), toDate32('2015-08-19')); 1 -- Non-const vs non-const columns -SELECT dateDiff('day', materialize(toDate32('1900-01-01')), materialize(toDate32('1900-01-02'))); +SELECT dateDiff('day', materialize(toDate32('1927-01-01')), materialize(toDate32('1927-01-02'))); 1 -SELECT dateDiff('day', materialize(toDate32('1900-01-01')), materialize(toDateTime64('1900-01-02 00:00:00', 3))); +SELECT dateDiff('day', materialize(toDate32('1927-01-01')), materialize(toDateTime64('1927-01-02 00:00:00', 3))); 1 -SELECT dateDiff('day', materialize(toDateTime64('1900-01-01 00:00:00', 3)), materialize(toDate32('1900-01-02'))); +SELECT dateDiff('day', materialize(toDateTime64('1927-01-01 00:00:00', 3)), materialize(toDate32('1927-01-02'))); 1 SELECT dateDiff('day', materialize(toDate32('2015-08-18')), materialize(toDateTime('2015-08-19 00:00:00'))); 1 diff --git a/tests/queries/0_stateless/02458_datediff_date32.sql b/tests/queries/0_stateless/02458_datediff_date32.sql index 4c26e04ac27..b4cb203080e 100644 --- a/tests/queries/0_stateless/02458_datediff_date32.sql +++ b/tests/queries/0_stateless/02458_datediff_date32.sql @@ -1,35 +1,35 @@ -- { echo } -- Date32 vs Date32 -SELECT dateDiff('second', toDate32('1900-01-01'), toDate32('1900-01-02')); -SELECT dateDiff('minute', toDate32('1900-01-01'), toDate32('1900-01-02')); -SELECT dateDiff('hour', toDate32('1900-01-01'), toDate32('1900-01-02')); -SELECT dateDiff('day', toDate32('1900-01-01'), toDate32('1900-01-02')); -SELECT dateDiff('week', toDate32('1900-01-01'), toDate32('1900-01-08')); -SELECT dateDiff('month', toDate32('1900-01-01'), toDate32('1900-02-01')); -SELECT dateDiff('quarter', toDate32('1900-01-01'), toDate32('1900-04-01')); -SELECT dateDiff('year', toDate32('1900-01-01'), toDate32('1901-01-01')); +SELECT dateDiff('second', toDate32('1927-01-01'), toDate32('1927-01-02')); +SELECT dateDiff('minute', toDate32('1927-01-01'), toDate32('1927-01-02')); +SELECT dateDiff('hour', toDate32('1927-01-01'), toDate32('1927-01-02')); +SELECT dateDiff('day', toDate32('1927-01-01'), toDate32('1927-01-02')); +SELECT dateDiff('week', toDate32('1927-01-01'), toDate32('1927-01-08')); +SELECT dateDiff('month', toDate32('1927-01-01'), toDate32('1927-02-01')); +SELECT dateDiff('quarter', toDate32('1927-01-01'), toDate32('1927-04-01')); +SELECT dateDiff('year', toDate32('1927-01-01'), toDate32('1928-01-01')); -- With DateTime64 -- Date32 vs DateTime64 -SELECT dateDiff('second', toDate32('1900-01-01'), toDateTime64('1900-01-02 00:00:00', 3)); -SELECT dateDiff('minute', toDate32('1900-01-01'), toDateTime64('1900-01-02 00:00:00', 3)); -SELECT dateDiff('hour', toDate32('1900-01-01'), toDateTime64('1900-01-02 00:00:00', 3)); -SELECT dateDiff('day', toDate32('1900-01-01'), toDateTime64('1900-01-02 00:00:00', 3)); -SELECT dateDiff('week', toDate32('1900-01-01'), toDateTime64('1900-01-08 00:00:00', 3)); -SELECT dateDiff('month', toDate32('1900-01-01'), toDateTime64('1900-02-01 00:00:00', 3)); -SELECT dateDiff('quarter', toDate32('1900-01-01'), toDateTime64('1900-04-01 00:00:00', 3)); -SELECT dateDiff('year', toDate32('1900-01-01'), toDateTime64('1901-01-01 00:00:00', 3)); +SELECT dateDiff('second', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); +SELECT dateDiff('minute', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); +SELECT dateDiff('hour', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); +SELECT dateDiff('day', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); +SELECT dateDiff('week', toDate32('1927-01-01'), toDateTime64('1927-01-08 00:00:00', 3)); +SELECT dateDiff('month', toDate32('1927-01-01'), toDateTime64('1927-02-01 00:00:00', 3)); +SELECT dateDiff('quarter', toDate32('1927-01-01'), toDateTime64('1927-04-01 00:00:00', 3)); +SELECT dateDiff('year', toDate32('1927-01-01'), toDateTime64('1928-01-01 00:00:00', 3)); -- DateTime64 vs Date32 -SELECT dateDiff('second', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-01-02')); -SELECT dateDiff('minute', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-01-02')); -SELECT dateDiff('hour', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-01-02')); -SELECT dateDiff('day', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-01-02')); -SELECT dateDiff('week', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-01-08')); -SELECT dateDiff('month', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-02-01')); -SELECT dateDiff('quarter', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1900-04-01')); -SELECT dateDiff('year', toDateTime64('1900-01-01 00:00:00', 3), toDate32('1901-01-01')); +SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); +SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); +SELECT dateDiff('hour', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); +SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); +SELECT dateDiff('week', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-08')); +SELECT dateDiff('month', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-02-01')); +SELECT dateDiff('quarter', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-04-01')); +SELECT dateDiff('year', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1928-01-01')); -- With DateTime -- Date32 vs DateTime @@ -74,27 +74,27 @@ SELECT dateDiff('quarter', toDate('2015-08-18'), toDate32('2015-11-18')); SELECT dateDiff('year', toDate('2015-08-18'), toDate32('2016-08-18')); -- Const vs non-const columns -SELECT dateDiff('day', toDate32('1900-01-01'), materialize(toDate32('1900-01-02'))); -SELECT dateDiff('day', toDate32('1900-01-01'), materialize(toDateTime64('1900-01-02 00:00:00', 3))); -SELECT dateDiff('day', toDateTime64('1900-01-01 00:00:00', 3), materialize(toDate32('1900-01-02'))); +SELECT dateDiff('day', toDate32('1927-01-01'), materialize(toDate32('1927-01-02'))); +SELECT dateDiff('day', toDate32('1927-01-01'), materialize(toDateTime64('1927-01-02 00:00:00', 3))); +SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 3), materialize(toDate32('1927-01-02'))); SELECT dateDiff('day', toDate32('2015-08-18'), materialize(toDateTime('2015-08-19 00:00:00'))); SELECT dateDiff('day', toDateTime('2015-08-18 00:00:00'), materialize(toDate32('2015-08-19'))); SELECT dateDiff('day', toDate32('2015-08-18'), materialize(toDate('2015-08-19'))); SELECT dateDiff('day', toDate('2015-08-18'), materialize(toDate32('2015-08-19'))); -- Non-const vs const columns -SELECT dateDiff('day', materialize(toDate32('1900-01-01')), toDate32('1900-01-02')); -SELECT dateDiff('day', materialize(toDate32('1900-01-01')), toDateTime64('1900-01-02 00:00:00', 3)); -SELECT dateDiff('day', materialize(toDateTime64('1900-01-01 00:00:00', 3)), toDate32('1900-01-02')); +SELECT dateDiff('day', materialize(toDate32('1927-01-01')), toDate32('1927-01-02')); +SELECT dateDiff('day', materialize(toDate32('1927-01-01')), toDateTime64('1927-01-02 00:00:00', 3)); +SELECT dateDiff('day', materialize(toDateTime64('1927-01-01 00:00:00', 3)), toDate32('1927-01-02')); SELECT dateDiff('day', materialize(toDate32('2015-08-18')), toDateTime('2015-08-19 00:00:00')); SELECT dateDiff('day', materialize(toDateTime('2015-08-18 00:00:00')), toDate32('2015-08-19')); SELECT dateDiff('day', materialize(toDate32('2015-08-18')), toDate('2015-08-19')); SELECT dateDiff('day', materialize(toDate('2015-08-18')), toDate32('2015-08-19')); -- Non-const vs non-const columns -SELECT dateDiff('day', materialize(toDate32('1900-01-01')), materialize(toDate32('1900-01-02'))); -SELECT dateDiff('day', materialize(toDate32('1900-01-01')), materialize(toDateTime64('1900-01-02 00:00:00', 3))); -SELECT dateDiff('day', materialize(toDateTime64('1900-01-01 00:00:00', 3)), materialize(toDate32('1900-01-02'))); +SELECT dateDiff('day', materialize(toDate32('1927-01-01')), materialize(toDate32('1927-01-02'))); +SELECT dateDiff('day', materialize(toDate32('1927-01-01')), materialize(toDateTime64('1927-01-02 00:00:00', 3))); +SELECT dateDiff('day', materialize(toDateTime64('1927-01-01 00:00:00', 3)), materialize(toDate32('1927-01-02'))); SELECT dateDiff('day', materialize(toDate32('2015-08-18')), materialize(toDateTime('2015-08-19 00:00:00'))); SELECT dateDiff('day', materialize(toDateTime('2015-08-18 00:00:00')), materialize(toDate32('2015-08-19'))); SELECT dateDiff('day', materialize(toDate32('2015-08-18')), materialize(toDate('2015-08-19'))); From 5dabbf89adac850ba042afd4151e463682a28892 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 24 Oct 2022 13:43:24 +0200 Subject: [PATCH 157/188] Fixing build. --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 33da39b5813..15e5fed1fff 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -230,7 +230,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files"); auto settings_ptr = s3_settings.get(); - ScheduleFunc scheduler; + ThreadPoolCallbackRunner scheduler; if (write_settings.s3_allow_parallel_part_upload) scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "VFSWrite"); From d0e64eb15bc457b56d24d39f6df63ee89f2f37e6 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 24 Oct 2022 16:14:05 +0200 Subject: [PATCH 158/188] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 5 +- src/Planner/Planner.cpp | 2 +- src/Planner/PlannerActionsVisitor.cpp | 13 +- src/Planner/PlannerJoins.cpp | 1 + .../02342_analyzer_compound_types.reference | 36 ----- .../02342_analyzer_compound_types.sql | 64 ++++----- .../02389_analyzer_nested_lambda.reference | 121 ++++++++++++++++ .../02389_analyzer_nested_lambda.sql | 129 ++++++++++++++++++ 8 files changed, 293 insertions(+), 78 deletions(-) create mode 100644 tests/queries/0_stateless/02389_analyzer_nested_lambda.reference create mode 100644 tests/queries/0_stateless/02389_analyzer_nested_lambda.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index de3dc434ebf..e55f296e1b7 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -3696,12 +3696,14 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto & function_argument = function_arguments[function_argument_index]; ColumnWithTypeAndName argument_column; + bool argument_is_lambda = false; /** If function argument is lambda, save lambda argument index and initialize argument type as DataTypeFunction * where function argument types are initialized with empty array of lambda arguments size. */ if (const auto * lambda_node = function_argument->as()) { + argument_is_lambda = true; size_t lambda_arguments_size = lambda_node->getArguments().getNodes().size(); argument_column.type = std::make_shared(DataTypes(lambda_arguments_size, nullptr), nullptr); function_lambda_arguments_indexes.push_back(function_argument_index); @@ -3724,7 +3726,8 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi function_node.getFunctionName(), scope.scope_node->formatASTForErrorMessage()); - if (const auto constant_value = function_argument->getConstantValueOrNull()) + const auto constant_value = function_argument->getConstantValueOrNull(); + if (!argument_is_lambda && constant_value) { argument_column.column = constant_value->getType()->createColumnConst(1, constant_value->getValue()); argument_column.type = constant_value->getType(); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 2e128c93d08..97f82d06463 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -267,7 +267,7 @@ void Planner::buildQueryPlanIfNeeded() if (union_mode == SelectUnionMode::UNION_DEFAULT || union_mode == SelectUnionMode::EXCEPT_DEFAULT || union_mode == SelectUnionMode::INTERSECT_DEFAULT) - throw Exception(ErrorCodes::LOGICAL_ERROR, "UNION mode must be initialized"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "UNION mode must be initialized"); size_t queries_size = union_query_tree->getQueries().getNodes().size(); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index d47ecb238fc..7bc53127101 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -297,8 +297,6 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi lambda_arguments_names_and_types.emplace_back(lambda_argument_name, std::move(lambda_argument_type)); } - size_t previous_scope_node_actions_stack_index = actions_stack.size() - 1; - auto lambda_actions_dag = std::make_shared(); actions_stack.emplace_back(lambda_actions_dag, node); @@ -313,16 +311,18 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi ActionsDAG::NodeRawConstPtrs lambda_children; Names required_column_names = lambda_actions->getRequiredColumns(); + if (level == actions_stack.size() - 1) + --level; + const auto & lambda_argument_names = lambda_node.getArgumentNames(); for (const auto & required_column_name : required_column_names) { - auto it = std::find_if( - lambda_argument_names.begin(), lambda_argument_names.end(), [&](auto & value) { return value == required_column_name; }); + auto it = std::find(lambda_argument_names.begin(), lambda_argument_names.end(), required_column_name); if (it == lambda_argument_names.end()) { - lambda_children.push_back(actions_stack[previous_scope_node_actions_stack_index].getNodeOrThrow(required_column_name)); + lambda_children.push_back(actions_stack[level].getNodeOrThrow(required_column_name)); captured_column_names.push_back(required_column_name); } } @@ -332,9 +332,6 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi lambda_actions, captured_column_names, lambda_arguments_names_and_types, result_type, lambda_expression_node_name); actions_stack.pop_back(); - if (level == actions_stack.size()) - --level; - actions_stack[level].addFunctionIfNecessary(lambda_node_name, std::move(lambda_children), std::move(function_capture)); size_t actions_stack_size = actions_stack.size(); diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index e64de3359e8..f62517eaaad 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -424,6 +424,7 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & left_key_node->result_type->getName(), right_key_node->result_name, right_key_node->result_type->getName()); + throw; } auto cast_type_name = common_type->getName(); diff --git a/tests/queries/0_stateless/02342_analyzer_compound_types.reference b/tests/queries/0_stateless/02342_analyzer_compound_types.reference index 439ff7d9f39..51e0bbe6e92 100644 --- a/tests/queries/0_stateless/02342_analyzer_compound_types.reference +++ b/tests/queries/0_stateless/02342_analyzer_compound_types.reference @@ -68,42 +68,6 @@ value_0_level_1 value_1_level_1 toString(value.value_0_level_0.value_0_level_1) String toString(value.value_0_level_0.value_1_level_1) String value_0_level_1 value_1_level_1 -Array of tuples -id UInt64 -value.value_0_level_0 Array(Tuple(value_0_level_1 String, value_1_level_1 String)) -value.value_1_level_0 Array(String) -0 [('value_0_level_1','value_1_level_1')] ['value_1_level_0'] --- -value.value_0_level_0 Array(Tuple(value_0_level_1 String, value_1_level_1 String)) -value.value_1_level_0 Array(String) -[('value_0_level_1','value_1_level_1')] ['value_1_level_0'] --- -value.value_0_level_0.value_0_level_1 Array(String) -value.value_0_level_0.value_1_level_1 Array(String) -['value_0_level_1'] ['value_1_level_1'] --- -alias_value Array(Tuple(value_0_level_1 String, value_1_level_1 String)) -alias_value.value_0_level_1 Array(String) -alias_value.value_1_level_1 Array(String) -[('value_0_level_1','value_1_level_1')] ['value_0_level_1'] ['value_1_level_1'] --- -alias_value Array(Tuple(value_0_level_1 String, value_1_level_1 String)) -alias_value.value_0_level_1 Array(String) -alias_value.value_1_level_1 Array(String) -[('value_0_level_1','value_1_level_1')] ['value_0_level_1'] ['value_1_level_1'] --- -alias_value Array(Tuple(value_0_level_1 String, value_1_level_1 String)) -toString(alias_value.value_0_level_1) String -toString(alias_value.value_1_level_1) String -[('value_0_level_1','value_1_level_1')] [\'value_0_level_1\'] [\'value_1_level_1\'] --- -value.value_0_level_0.value_0_level_1 Array(String) -value.value_0_level_0.value_1_level_1 Array(String) -['value_0_level_1'] ['value_1_level_1'] --- -toString(value.value_0_level_0.value_0_level_1) String -toString(value.value_0_level_0.value_1_level_1) String -[\'value_0_level_1\'] [\'value_1_level_1\'] Nested id UInt64 value.value_0_level_0 Array(Nested(value_0_level_1 String, value_1_level_1 String)) diff --git a/tests/queries/0_stateless/02342_analyzer_compound_types.sql b/tests/queries/0_stateless/02342_analyzer_compound_types.sql index 6752064d13f..6b4f323fd66 100644 --- a/tests/queries/0_stateless/02342_analyzer_compound_types.sql +++ b/tests/queries/0_stateless/02342_analyzer_compound_types.sql @@ -92,56 +92,56 @@ SELECT value.value_0_level_0.* APPLY toString FROM test_table; DROP TABLE test_table; -SELECT 'Array of tuples'; +-- SELECT 'Array of tuples'; -DROP TABLE IF EXISTS test_table; -CREATE TABLE test_table -( - id UInt64, - value Array(Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String)) -) ENGINE=MergeTree ORDER BY id; +-- DROP TABLE IF EXISTS test_table; +-- CREATE TABLE test_table +-- ( +-- id UInt64, +-- value Array(Tuple(value_0_level_0 Tuple(value_0_level_1 String, value_1_level_1 String), value_1_level_0 String)) +-- ) ENGINE=MergeTree ORDER BY id; -INSERT INTO test_table VALUES (0, [('value_0_level_1', 'value_1_level_1')], ['value_1_level_0']); +-- INSERT INTO test_table VALUES (0, [('value_0_level_1', 'value_1_level_1')], ['value_1_level_0']); -DESCRIBE (SELECT * FROM test_table); -SELECT * FROM test_table; +-- DESCRIBE (SELECT * FROM test_table); +-- SELECT * FROM test_table; -SELECT '--'; +-- SELECT '--'; -DESCRIBE (SELECT value.value_0_level_0, value.value_1_level_0 FROM test_table); -SELECT value.value_0_level_0, value.value_1_level_0 FROM test_table; +-- DESCRIBE (SELECT value.value_0_level_0, value.value_1_level_0 FROM test_table); +-- SELECT value.value_0_level_0, value.value_1_level_0 FROM test_table; -SELECT '--'; +-- SELECT '--'; -DESCRIBE (SELECT value.value_0_level_0.value_0_level_1, value.value_0_level_0.value_1_level_1 FROM test_table); -SELECT value.value_0_level_0.value_0_level_1, value.value_0_level_0.value_1_level_1 FROM test_table; +-- DESCRIBE (SELECT value.value_0_level_0.value_0_level_1, value.value_0_level_0.value_1_level_1 FROM test_table); +-- SELECT value.value_0_level_0.value_0_level_1, value.value_0_level_0.value_1_level_1 FROM test_table; -SELECT '--'; +-- SELECT '--'; -DESCRIBE (SELECT value.value_0_level_0 AS alias_value, alias_value.value_0_level_1, alias_value.value_1_level_1 FROM test_table); -SELECT value.value_0_level_0 AS alias_value, alias_value.value_0_level_1, alias_value.value_1_level_1 FROM test_table; +-- DESCRIBE (SELECT value.value_0_level_0 AS alias_value, alias_value.value_0_level_1, alias_value.value_1_level_1 FROM test_table); +-- SELECT value.value_0_level_0 AS alias_value, alias_value.value_0_level_1, alias_value.value_1_level_1 FROM test_table; -SELECT '--'; +-- SELECT '--'; -DESCRIBE (SELECT value.value_0_level_0 AS alias_value, alias_value.* FROM test_table); -SELECT value.value_0_level_0 AS alias_value, alias_value.* FROM test_table; +-- DESCRIBE (SELECT value.value_0_level_0 AS alias_value, alias_value.* FROM test_table); +-- SELECT value.value_0_level_0 AS alias_value, alias_value.* FROM test_table; -SELECT '--'; +-- SELECT '--'; -DESCRIBE (SELECT value.value_0_level_0 AS alias_value, alias_value.* APPLY toString FROM test_table); -SELECT value.value_0_level_0 AS alias_value, alias_value.* APPLY toString FROM test_table; +-- DESCRIBE (SELECT value.value_0_level_0 AS alias_value, alias_value.* APPLY toString FROM test_table); +-- SELECT value.value_0_level_0 AS alias_value, alias_value.* APPLY toString FROM test_table; -SELECT '--'; +-- SELECT '--'; -DESCRIBE (SELECT value.value_0_level_0.* FROM test_table); -SELECT value.value_0_level_0.* FROM test_table; +-- DESCRIBE (SELECT value.value_0_level_0.* FROM test_table); +-- SELECT value.value_0_level_0.* FROM test_table; -SELECT '--'; +-- SELECT '--'; -DESCRIBE (SELECT value.value_0_level_0.* APPLY toString FROM test_table); -SELECT value.value_0_level_0.* APPLY toString FROM test_table; +-- DESCRIBE (SELECT value.value_0_level_0.* APPLY toString FROM test_table); +-- SELECT value.value_0_level_0.* APPLY toString FROM test_table; -DROP TABLE test_table; +-- DROP TABLE test_table; SELECT 'Nested'; diff --git a/tests/queries/0_stateless/02389_analyzer_nested_lambda.reference b/tests/queries/0_stateless/02389_analyzer_nested_lambda.reference new file mode 100644 index 00000000000..935c53358c0 --- /dev/null +++ b/tests/queries/0_stateless/02389_analyzer_nested_lambda.reference @@ -0,0 +1,121 @@ +-- { echoOn } + +SELECT arrayMap(x -> x + arrayMap(x -> x + 1, [1])[1], [1,2,3]); +[3,4,5] +SELECT '--'; +-- +SELECT arrayMap(x -> x + arrayMap(x -> 5, [1])[1], [1,2,3]); +[6,7,8] +SELECT '--'; +-- +SELECT 5 AS constant, arrayMap(x -> x + arrayMap(x -> constant, [1])[1], [1,2,3]); +5 [6,7,8] +SELECT '--'; +-- +SELECT arrayMap(x -> x + arrayMap(x -> x, [1])[1], [1,2,3]); +[2,3,4] +SELECT '--'; +-- +SELECT arrayMap(x -> x + arrayMap(y -> x + y, [1])[1], [1,2,3]); +[3,5,7] +SELECT '--'; +-- +SELECT arrayMap(x -> x + arrayMap(x -> (SELECT 5), [1])[1], [1,2,3]); +[6,7,8] +SELECT '--'; +-- +SELECT (SELECT 5) AS subquery, arrayMap(x -> x + arrayMap(x -> subquery, [1])[1], [1,2,3]); +5 [6,7,8] +SELECT '--'; +-- +SELECT arrayMap(x -> x + arrayMap(x -> (SELECT 5 UNION DISTINCT SELECT 5), [1])[1], [1,2,3]); +[6,7,8] +SELECT '--'; +-- +SELECT (SELECT 5 UNION DISTINCT SELECT 5) AS subquery, arrayMap(x -> x + arrayMap(x -> subquery, [1])[1], [1,2,3]); +5 [6,7,8] +SELECT '--'; +-- +WITH x -> toString(x) AS lambda SELECT arrayMap(x -> lambda(x), [1,2,3]); +['1','2','3'] +SELECT '--'; +-- +WITH x -> toString(x) AS lambda SELECT arrayMap(x -> arrayMap(y -> concat(lambda(x), '_', lambda(y)), [1,2,3]), [1,2,3]); +[['1_1','1_2','1_3'],['2_1','2_2','2_3'],['3_1','3_2','3_3']] +SELECT '--'; +-- +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; +INSERT INTO test_table VALUES (0, 'Value'); +SELECT arrayMap(x -> x + arrayMap(x -> id, [1])[1], [1,2,3]) FROM test_table; +[1,2,3] +SELECT '--'; +-- +SELECT arrayMap(x -> x + arrayMap(x -> x + id, [1])[1], [1,2,3]) FROM test_table; +[2,3,4] +SELECT '--'; +-- +SELECT arrayMap(x -> x + arrayMap(y -> x + y + id, [1])[1], [1,2,3]) FROM test_table; +[3,5,7] +SELECT '--'; +-- +SELECT id AS id_alias, arrayMap(x -> x + arrayMap(y -> x + y + id_alias, [1])[1], [1,2,3]) FROM test_table; +0 [3,5,7] +SELECT '--'; +-- +SELECT arrayMap(x -> x + arrayMap(x -> 5, [1])[1], [1,2,3]) FROM test_table; +[6,7,8] +SELECT '--'; +-- +SELECT 5 AS constant, arrayMap(x -> x + arrayMap(x -> constant, [1])[1], [1,2,3]) FROM test_table; +5 [6,7,8] +SELECT '--'; +-- +SELECT 5 AS constant, arrayMap(x -> x + arrayMap(x -> x + constant, [1])[1], [1,2,3]) FROM test_table; +5 [7,8,9] +SELECT '--'; +-- +SELECT 5 AS constant, arrayMap(x -> x + arrayMap(x -> x + id + constant, [1])[1], [1,2,3]) FROM test_table; +5 [7,8,9] +SELECT '--'; +-- +SELECT 5 AS constant, arrayMap(x -> x + arrayMap(y -> x + y + id + constant, [1])[1], [1,2,3]) FROM test_table; +5 [8,10,12] +SELECT '--'; +-- +SELECT arrayMap(x -> x + arrayMap(x -> id + (SELECT id FROM test_table), [1])[1], [1,2,3]) FROM test_table; +[1,2,3] +SELECT '--'; +-- +SELECT arrayMap(x -> id + arrayMap(x -> id + (SELECT id FROM test_table), [1])[1], [1,2,3]) FROM test_table; +[0,0,0] +SELECT '--'; +-- +SELECT arrayMap(x -> id + arrayMap(x -> id + (SELECT id FROM test_table UNION DISTINCT SELECT id FROM test_table), [1])[1], [1,2,3]) FROM test_table; +[0,0,0] +SELECT '--'; +-- +WITH x -> toString(id) AS lambda SELECT arrayMap(x -> lambda(x), [1,2,3]) FROM test_table; +['0','0','0'] +SELECT '--'; +-- +WITH x -> toString(id) AS lambda SELECT arrayMap(x -> arrayMap(y -> lambda(y), [1,2,3]), [1,2,3]) FROM test_table; +[['0','0','0'],['0','0','0'],['0','0','0']] +SELECT '--'; +-- +WITH x -> toString(id) AS lambda SELECT arrayMap(x -> arrayMap(y -> concat(lambda(x), '_', lambda(y)), [1,2,3]), [1,2,3]) FROM test_table; +[['0_0','0_0','0_0'],['0_0','0_0','0_0'],['0_0','0_0','0_0']] +SELECT '--'; +-- +SELECT arrayMap(x -> concat(concat(concat(concat(concat(toString(id), '___\0_______\0____'), toString(id), concat(concat(toString(id), ''), toString(id)), toString(id)), + arrayMap(x -> concat(concat(concat(concat(toString(id), ''), toString(id)), toString(id), '___\0_______\0____'), toString(id)) AS lambda, [NULL, inf, 1, 1]), + concat(toString(id), NULL), toString(id)), toString(id))) AS lambda, [NULL, NULL, 2147483647]) +FROM test_table WHERE concat(concat(concat(toString(id), '___\0_______\0____'), toString(id)), concat(toString(id), NULL), toString(id)); +SELECT '--'; +-- +SELECT arrayMap(x -> concat(toString(id), arrayMap(x -> toString(1), [NULL])), [NULL]) FROM test_table; -- { serverError 44 }; +DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql b/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql new file mode 100644 index 00000000000..3be07a35425 --- /dev/null +++ b/tests/queries/0_stateless/02389_analyzer_nested_lambda.sql @@ -0,0 +1,129 @@ +SET use_analyzer = 1; + +-- { echoOn } + +SELECT arrayMap(x -> x + arrayMap(x -> x + 1, [1])[1], [1,2,3]); + +SELECT '--'; + +SELECT arrayMap(x -> x + arrayMap(x -> 5, [1])[1], [1,2,3]); + +SELECT '--'; + +SELECT 5 AS constant, arrayMap(x -> x + arrayMap(x -> constant, [1])[1], [1,2,3]); + +SELECT '--'; + +SELECT arrayMap(x -> x + arrayMap(x -> x, [1])[1], [1,2,3]); + +SELECT '--'; + +SELECT arrayMap(x -> x + arrayMap(y -> x + y, [1])[1], [1,2,3]); + +SELECT '--'; + +SELECT arrayMap(x -> x + arrayMap(x -> (SELECT 5), [1])[1], [1,2,3]); + +SELECT '--'; + +SELECT (SELECT 5) AS subquery, arrayMap(x -> x + arrayMap(x -> subquery, [1])[1], [1,2,3]); + +SELECT '--'; + +SELECT arrayMap(x -> x + arrayMap(x -> (SELECT 5 UNION DISTINCT SELECT 5), [1])[1], [1,2,3]); + +SELECT '--'; + +SELECT (SELECT 5 UNION DISTINCT SELECT 5) AS subquery, arrayMap(x -> x + arrayMap(x -> subquery, [1])[1], [1,2,3]); + +SELECT '--'; + +WITH x -> toString(x) AS lambda SELECT arrayMap(x -> lambda(x), [1,2,3]); + +SELECT '--'; + +WITH x -> toString(x) AS lambda SELECT arrayMap(x -> arrayMap(y -> concat(lambda(x), '_', lambda(y)), [1,2,3]), [1,2,3]); + +SELECT '--'; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +SELECT arrayMap(x -> x + arrayMap(x -> id, [1])[1], [1,2,3]) FROM test_table; + +SELECT '--'; + +SELECT arrayMap(x -> x + arrayMap(x -> x + id, [1])[1], [1,2,3]) FROM test_table; + +SELECT '--'; + +SELECT arrayMap(x -> x + arrayMap(y -> x + y + id, [1])[1], [1,2,3]) FROM test_table; + +SELECT '--'; + +SELECT id AS id_alias, arrayMap(x -> x + arrayMap(y -> x + y + id_alias, [1])[1], [1,2,3]) FROM test_table; + +SELECT '--'; + +SELECT arrayMap(x -> x + arrayMap(x -> 5, [1])[1], [1,2,3]) FROM test_table; + +SELECT '--'; + +SELECT 5 AS constant, arrayMap(x -> x + arrayMap(x -> constant, [1])[1], [1,2,3]) FROM test_table; + +SELECT '--'; + +SELECT 5 AS constant, arrayMap(x -> x + arrayMap(x -> x + constant, [1])[1], [1,2,3]) FROM test_table; + +SELECT '--'; + +SELECT 5 AS constant, arrayMap(x -> x + arrayMap(x -> x + id + constant, [1])[1], [1,2,3]) FROM test_table; + +SELECT '--'; + +SELECT 5 AS constant, arrayMap(x -> x + arrayMap(y -> x + y + id + constant, [1])[1], [1,2,3]) FROM test_table; + +SELECT '--'; + +SELECT arrayMap(x -> x + arrayMap(x -> id + (SELECT id FROM test_table), [1])[1], [1,2,3]) FROM test_table; + +SELECT '--'; + +SELECT arrayMap(x -> id + arrayMap(x -> id + (SELECT id FROM test_table), [1])[1], [1,2,3]) FROM test_table; + +SELECT '--'; + +SELECT arrayMap(x -> id + arrayMap(x -> id + (SELECT id FROM test_table UNION DISTINCT SELECT id FROM test_table), [1])[1], [1,2,3]) FROM test_table; + +SELECT '--'; + +WITH x -> toString(id) AS lambda SELECT arrayMap(x -> lambda(x), [1,2,3]) FROM test_table; + +SELECT '--'; + +WITH x -> toString(id) AS lambda SELECT arrayMap(x -> arrayMap(y -> lambda(y), [1,2,3]), [1,2,3]) FROM test_table; + +SELECT '--'; + +WITH x -> toString(id) AS lambda SELECT arrayMap(x -> arrayMap(y -> concat(lambda(x), '_', lambda(y)), [1,2,3]), [1,2,3]) FROM test_table; + +SELECT '--'; + +SELECT arrayMap(x -> concat(concat(concat(concat(concat(toString(id), '___\0_______\0____'), toString(id), concat(concat(toString(id), ''), toString(id)), toString(id)), + arrayMap(x -> concat(concat(concat(concat(toString(id), ''), toString(id)), toString(id), '___\0_______\0____'), toString(id)) AS lambda, [NULL, inf, 1, 1]), + concat(toString(id), NULL), toString(id)), toString(id))) AS lambda, [NULL, NULL, 2147483647]) +FROM test_table WHERE concat(concat(concat(toString(id), '___\0_______\0____'), toString(id)), concat(toString(id), NULL), toString(id)); + +SELECT '--'; + +SELECT arrayMap(x -> concat(toString(id), arrayMap(x -> toString(1), [NULL])), [NULL]) FROM test_table; -- { serverError 44 }; + +DROP TABLE test_table; + +-- { echoOff } From beb40e45791105bf7a76e06f82d12223b5c35f7a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 24 Oct 2022 14:22:06 +0000 Subject: [PATCH 159/188] Split tests. --- .../integration/test_s3_zero_copy_ttl/test.py | 54 ----------------- .../test_ttl_move_memory_usage.py | 60 +++++++++++++++++++ 2 files changed, 60 insertions(+), 54 deletions(-) create mode 100644 tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py diff --git a/tests/integration/test_s3_zero_copy_ttl/test.py b/tests/integration/test_s3_zero_copy_ttl/test.py index 843c216ce72..9a782aacef6 100644 --- a/tests/integration/test_s3_zero_copy_ttl/test.py +++ b/tests/integration/test_s3_zero_copy_ttl/test.py @@ -15,11 +15,6 @@ node3 = cluster.add_instance( "node3", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True ) -single_node_cluster = ClickHouseCluster(__file__) -small_node = single_node_cluster.add_instance( - "small_node", main_configs=["configs/s3.xml"], with_minio=True -) - @pytest.fixture(scope="module") def started_cluster(): @@ -97,52 +92,3 @@ def test_ttl_move_and_s3(started_cluster): print(f"Attempts remaining: {attempt}") assert counter == 300 - - -@pytest.fixture(scope="module") -def started_single_node_cluster(): - try: - single_node_cluster.start() - - yield single_node_cluster - finally: - single_node_cluster.shutdown() - - -def test_move_and_s3_memory_usage(started_single_node_cluster): - if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): - pytest.skip("Disabled for debug and sanitizers. Too slow.") - - small_node.query( - "CREATE TABLE s3_test_with_ttl (x UInt32, a String codec(NONE), b String codec(NONE), c String codec(NONE), d String codec(NONE), e String codec(NONE)) engine = MergeTree order by x partition by x SETTINGS storage_policy='s3_and_default'" - ) - - for _ in range(10): - small_node.query( - "insert into s3_test_with_ttl select 0, repeat('a', 100), repeat('b', 100), repeat('c', 100), repeat('d', 100), repeat('e', 100) from zeros(400000) settings max_block_size = 8192, max_insert_block_size=10000000, min_insert_block_size_rows=10000000" - ) - - # After this, we should have 5 columns per 10 * 100 * 400000 ~ 400 MB; total ~2G data in partition - small_node.query("optimize table s3_test_with_ttl final") - - small_node.query("system flush logs") - # Will take memory usage from metric_log. - # It is easier then specifying total memory limit (insert queries can hit this limit). - small_node.query("truncate table system.metric_log") - - small_node.query( - "alter table s3_test_with_ttl move partition 0 to volume 'external'", - settings={"send_logs_level": "error"}, - ) - small_node.query("system flush logs") - max_usage = small_node.query( - "select max(CurrentMetric_MemoryTracking) from system.metric_log" - ) - # 3G limit is a big one. However, we can hit it anyway with parallel s3 writes enabled. - # Also actual value can be bigger because of memory drift. - # Increase it a little bit if test fails. - assert int(max_usage) < 3e9 - res = small_node.query( - "select * from system.errors where last_error_message like '%Memory limit%' limit 1" - ) - assert res == "" diff --git a/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py b/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py new file mode 100644 index 00000000000..a1e10cde031 --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py @@ -0,0 +1,60 @@ +#!/usr/bin/env python3 +import time + +import pytest +from helpers.cluster import ClickHouseCluster + + +single_node_cluster = ClickHouseCluster(__file__) +small_node = single_node_cluster.add_instance( + "small_node", main_configs=["configs/s3.xml"], with_minio=True +) + + +@pytest.fixture(scope="module") +def started_single_node_cluster(): + try: + single_node_cluster.start() + + yield single_node_cluster + finally: + single_node_cluster.shutdown() + + +def test_move_and_s3_memory_usage(started_single_node_cluster): + if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): + pytest.skip("Disabled for debug and sanitizers. Too slow.") + + small_node.query( + "CREATE TABLE s3_test_with_ttl (x UInt32, a String codec(NONE), b String codec(NONE), c String codec(NONE), d String codec(NONE), e String codec(NONE)) engine = MergeTree order by x partition by x SETTINGS storage_policy='s3_and_default'" + ) + + for _ in range(10): + small_node.query( + "insert into s3_test_with_ttl select 0, repeat('a', 100), repeat('b', 100), repeat('c', 100), repeat('d', 100), repeat('e', 100) from zeros(400000) settings max_block_size = 8192, max_insert_block_size=10000000, min_insert_block_size_rows=10000000" + ) + + # After this, we should have 5 columns per 10 * 100 * 400000 ~ 400 MB; total ~2G data in partition + small_node.query("optimize table s3_test_with_ttl final") + + small_node.query("system flush logs") + # Will take memory usage from metric_log. + # It is easier then specifying total memory limit (insert queries can hit this limit). + small_node.query("truncate table system.metric_log") + + small_node.query( + "alter table s3_test_with_ttl move partition 0 to volume 'external'", + settings={"send_logs_level": "error"}, + ) + small_node.query("system flush logs") + max_usage = small_node.query( + "select max(CurrentMetric_MemoryTracking) from system.metric_log" + ) + # 3G limit is a big one. However, we can hit it anyway with parallel s3 writes enabled. + # Also actual value can be bigger because of memory drift. + # Increase it a little bit if test fails. + assert int(max_usage) < 3e9 + res = small_node.query( + "select * from system.errors where last_error_message like '%Memory limit%' limit 1" + ) + assert res == "" From d5bd905a4186075c701520972cc5a7bce0e41f8e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 24 Oct 2022 15:09:35 +0000 Subject: [PATCH 160/188] Try to fix data race on zookeeper vs DDLWorker at server shutdown. --- src/Interpreters/Context.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 721d701c9a2..1274551a612 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -463,6 +463,18 @@ struct ContextSharedPart : boost::noncopyable std::unique_ptr delete_ddl_worker; std::unique_ptr delete_access_control; + /// Delete DDLWorker before zookeeper. + /// Cause it can call Context::getZooKeeper and ressurect it. + + { + auto lock = std::lock_guard(mutex); + delete_ddl_worker = std::move(ddl_worker); + } + + /// DDLWorker should be deleted without lock, cause its internal thread can + /// take it as well, which will cause deadlock. + delete_ddl_worker.reset(); + { auto lock = std::lock_guard(mutex); @@ -499,7 +511,6 @@ struct ContextSharedPart : boost::noncopyable delete_schedule_pool = std::move(schedule_pool); delete_distributed_schedule_pool = std::move(distributed_schedule_pool); delete_message_broker_schedule_pool = std::move(message_broker_schedule_pool); - delete_ddl_worker = std::move(ddl_worker); delete_access_control = std::move(access_control); /// Stop trace collector if any @@ -528,7 +539,6 @@ struct ContextSharedPart : boost::noncopyable delete_schedule_pool.reset(); delete_distributed_schedule_pool.reset(); delete_message_broker_schedule_pool.reset(); - delete_ddl_worker.reset(); delete_access_control.reset(); total_memory_tracker.resetOvercommitTracker(); From 93310fe8b188d3772aede5dd6a2c2208bbd814d2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 24 Oct 2022 15:51:21 +0000 Subject: [PATCH 161/188] Fix typo. --- src/Interpreters/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1274551a612..4b13100b582 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -464,7 +464,7 @@ struct ContextSharedPart : boost::noncopyable std::unique_ptr delete_access_control; /// Delete DDLWorker before zookeeper. - /// Cause it can call Context::getZooKeeper and ressurect it. + /// Cause it can call Context::getZooKeeper and resurrect it. { auto lock = std::lock_guard(mutex); From 9532b6024bcfc81c57c7064a702b6ed587e76afd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 19:02:05 +0300 Subject: [PATCH 162/188] Update CHANGELOG.md Co-authored-by: Dan Roscigno --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f2b8e9004b0..ea568985d31 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -42,7 +42,7 @@ * ColumnVector: optimize UInt8 index with AVX512VBMI. [#41247](https://github.com/ClickHouse/ClickHouse/pull/41247) ([Guo Wangyang](https://github.com/guowangy)). * Optimize the lock contentions for `ThreadGroupStatus::mutex`. The performance experiments of **SSB** (Star Schema Benchmark) on the ICX device (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads) shows that this change could bring a **2.95x** improvement of the geomean of all subcases' QPS. [#41675](https://github.com/ClickHouse/ClickHouse/pull/41675) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). * Add `ldapr` capabilities to AArch64 builds. This is supported from Graviton 2+, Azure and GCP instances. Only appeared in clang-15 [not so long ago](https://github.com/llvm/llvm-project/commit/9609b5daffe9fd28d83d83da895abc5113f76c24). [#41778](https://github.com/ClickHouse/ClickHouse/pull/41778) ([Daniel Kutenin](https://github.com/danlark1)). -* Improve performance when comparing strings and one argument is empty constant string. [#41870](https://github.com/ClickHouse/ClickHouse/pull/41870) ([Jiebin Sun](https://github.com/jiebinn)). +* Improve performance when comparing strings and one argument is an empty constant string. [#41870](https://github.com/ClickHouse/ClickHouse/pull/41870) ([Jiebin Sun](https://github.com/jiebinn)). * Optimize `insertFrom` of ColumnAggregateFunction to share Aggregate State in some cases. [#41960](https://github.com/ClickHouse/ClickHouse/pull/41960) ([flynn](https://github.com/ucasfl)). * Make writing to `azure_blob_storage` disks faster (respect `max_single_part_upload_size` instead of writing a block per each buffer size). Inefficiency mentioned in [#41754](https://github.com/ClickHouse/ClickHouse/issues/41754). [#42041](https://github.com/ClickHouse/ClickHouse/pull/42041) ([Kseniia Sumarokova](https://github.com/kssenii)). * Make thread ids in the process list and query_log unique to avoid waste. [#42180](https://github.com/ClickHouse/ClickHouse/pull/42180) ([Alexey Milovidov](https://github.com/alexey-milovidov)). From 182925071119174f783529f90ebcb09cda4dd4f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 19:02:16 +0300 Subject: [PATCH 163/188] Update CHANGELOG.md Co-authored-by: Dan Roscigno --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ea568985d31..192dafd20ff 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -49,7 +49,7 @@ * Support skipping cache completely (both download to cache and reading cached data) in case the requested read range exceeds the threshold defined by cache setting `bypass_cache_threashold`, requires to be enabled with `enable_bypass_cache_with_threshold`). [#42418](https://github.com/ClickHouse/ClickHouse/pull/42418) ([Han Shukai](https://github.com/KinderRiven)). This helps on slow local disks. #### Improvement -* Add setting `allow_implicit_no_password`: with combination with `allow_no_password` it forbids creating a user with no password unless `IDENTIFIED WITH no_password` is explicitly specified. [#41341](https://github.com/ClickHouse/ClickHouse/pull/41341) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add setting `allow_implicit_no_password`: in combination with `allow_no_password` it forbids creating a user with no password unless `IDENTIFIED WITH no_password` is explicitly specified. [#41341](https://github.com/ClickHouse/ClickHouse/pull/41341) ([Nikolay Degterinsky](https://github.com/evillique)). * Embedded Keeper will always start in the background allowing ClickHouse to start without achieving quorum. [#40991](https://github.com/ClickHouse/ClickHouse/pull/40991) ([Antonio Andelic](https://github.com/antonio2368)). * Made reestablishing a new connection to ZooKeeper more reactive in case of expiration of the previous one. Previously there was a task which spawns every minute by default and thus a table could be in readonly state for about this time. [#41092](https://github.com/ClickHouse/ClickHouse/pull/41092) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Now projections can be used with zero copy replication (zero-copy replication is a non-production feature). [#41147](https://github.com/ClickHouse/ClickHouse/pull/41147) ([alesapin](https://github.com/alesapin)). From 440d302a8c5c286a7394bd747f3b0af16a427bff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 19:02:28 +0300 Subject: [PATCH 164/188] Update CHANGELOG.md Co-authored-by: Dan Roscigno --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 192dafd20ff..8049037c99b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -54,7 +54,7 @@ * Made reestablishing a new connection to ZooKeeper more reactive in case of expiration of the previous one. Previously there was a task which spawns every minute by default and thus a table could be in readonly state for about this time. [#41092](https://github.com/ClickHouse/ClickHouse/pull/41092) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Now projections can be used with zero copy replication (zero-copy replication is a non-production feature). [#41147](https://github.com/ClickHouse/ClickHouse/pull/41147) ([alesapin](https://github.com/alesapin)). * Support expression `(EXPLAIN SELECT ...)` in a subquery. Queries like `SELECT * FROM (EXPLAIN PIPELINE SELECT col FROM TABLE ORDER BY col)` became valid. [#40630](https://github.com/ClickHouse/ClickHouse/pull/40630) ([Vladimir C](https://github.com/vdimir)). -* Allow changing `async_insert_max_data_size` or `async_insert_busy_timeout_ms` in scope of query. E.g. user wants to insert data rarely and she doesn't have an access to server config to tune default settings. [#40668](https://github.com/ClickHouse/ClickHouse/pull/40668) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Allow changing `async_insert_max_data_size` or `async_insert_busy_timeout_ms` in scope of query. E.g. user wants to insert data rarely and she doesn't have access to the server config to tune default settings. [#40668](https://github.com/ClickHouse/ClickHouse/pull/40668) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Improvements for reading from remote filesystems, made threadpool size for reads/writes configurable. Closes [#41070](https://github.com/ClickHouse/ClickHouse/issues/41070). [#41011](https://github.com/ClickHouse/ClickHouse/pull/41011) ([Kseniia Sumarokova](https://github.com/kssenii)). * Support all combinators combination in WindowTransform/arratReduce*/initializeAggregation/aggregate functions versioning. Previously combinators like `ForEach/Resample/Map` didn't work in these places, using them led to exception like`State function ... inserts results into non-state column`. [#41107](https://github.com/ClickHouse/ClickHouse/pull/41107) ([Kruglov Pavel](https://github.com/Avogar)). * Add function `tryDecrypt` that returns NULL when decrypt fail (e.g. decrypt with incorrect key) instead of throwing exception. [#41206](https://github.com/ClickHouse/ClickHouse/pull/41206) ([Duc Canh Le](https://github.com/canhld94)). From 6d255b3430d3d441803ed2be9570c73517040cc4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 19:02:41 +0300 Subject: [PATCH 165/188] Update CHANGELOG.md Co-authored-by: Dan Roscigno --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8049037c99b..e71dd5dd737 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -57,7 +57,7 @@ * Allow changing `async_insert_max_data_size` or `async_insert_busy_timeout_ms` in scope of query. E.g. user wants to insert data rarely and she doesn't have access to the server config to tune default settings. [#40668](https://github.com/ClickHouse/ClickHouse/pull/40668) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Improvements for reading from remote filesystems, made threadpool size for reads/writes configurable. Closes [#41070](https://github.com/ClickHouse/ClickHouse/issues/41070). [#41011](https://github.com/ClickHouse/ClickHouse/pull/41011) ([Kseniia Sumarokova](https://github.com/kssenii)). * Support all combinators combination in WindowTransform/arratReduce*/initializeAggregation/aggregate functions versioning. Previously combinators like `ForEach/Resample/Map` didn't work in these places, using them led to exception like`State function ... inserts results into non-state column`. [#41107](https://github.com/ClickHouse/ClickHouse/pull/41107) ([Kruglov Pavel](https://github.com/Avogar)). -* Add function `tryDecrypt` that returns NULL when decrypt fail (e.g. decrypt with incorrect key) instead of throwing exception. [#41206](https://github.com/ClickHouse/ClickHouse/pull/41206) ([Duc Canh Le](https://github.com/canhld94)). +* Add function `tryDecrypt` that returns NULL when decrypt fails (e.g. decrypt with incorrect key) instead of throwing an exception. [#41206](https://github.com/ClickHouse/ClickHouse/pull/41206) ([Duc Canh Le](https://github.com/canhld94)). * Add the `unreserved_space` column to the `system.disks` table to check how much space is not taken by reservations per disk. [#41254](https://github.com/ClickHouse/ClickHouse/pull/41254) ([filimonov](https://github.com/filimonov)). * Support s3 authorization headers in table function arguments. [#41261](https://github.com/ClickHouse/ClickHouse/pull/41261) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add support for MultiRead in Keeper and internal ZooKeeper client (this is an extension to ZooKeeper protocol, only available in ClickHouse Keeper). [#41410](https://github.com/ClickHouse/ClickHouse/pull/41410) ([Antonio Andelic](https://github.com/antonio2368)). From 4d490fa088f7e639f579b234833aadabdc4ca15f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 19:02:49 +0300 Subject: [PATCH 166/188] Update CHANGELOG.md Co-authored-by: Dan Roscigno --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e71dd5dd737..85363153e77 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -63,7 +63,7 @@ * Add support for MultiRead in Keeper and internal ZooKeeper client (this is an extension to ZooKeeper protocol, only available in ClickHouse Keeper). [#41410](https://github.com/ClickHouse/ClickHouse/pull/41410) ([Antonio Andelic](https://github.com/antonio2368)). * Add support for decimal type comparing with floating point literal in IN operator. [#41544](https://github.com/ClickHouse/ClickHouse/pull/41544) ([liang.huang](https://github.com/lhuang09287750)). * Allow readable size values (like `1TB`) in cache config. [#41688](https://github.com/ClickHouse/ClickHouse/pull/41688) ([Kseniia Sumarokova](https://github.com/kssenii)). -* ClickHouse could cache stale DNS entries for some period of time (15 seconds by default) until the cache won't be updated asynchronously. During these period ClickHouse can nevertheless try to establish a connection and produce errors. This behaviour is fixed. [#41707](https://github.com/ClickHouse/ClickHouse/pull/41707) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* ClickHouse could cache stale DNS entries for some period of time (15 seconds by default) until the cache won't be updated asynchronously. During these periods ClickHouse can nevertheless try to establish a connection and produce errors. This behavior is fixed. [#41707](https://github.com/ClickHouse/ClickHouse/pull/41707) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Add interactive history search with fzf-like utility (fzf/sk) for `clickhouse-client`/`clickhouse-local` (note you can use `FZF_DEFAULT_OPTS`/`SKIM_DEFAULT_OPTIONS` to additionally configure the behavior). [#41730](https://github.com/ClickHouse/ClickHouse/pull/41730) ([Azat Khuzhin](https://github.com/azat)). * For client when connecting to a secure server with invalid certificate only allow to proceed with '--accept-certificate' flag. [#41743](https://github.com/ClickHouse/ClickHouse/pull/41743) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). * Add function `tryBase58Decode`, similar to the existing function `tryBase64Decode`. [#41824](https://github.com/ClickHouse/ClickHouse/pull/41824) ([Robert Schulze](https://github.com/rschu1ze)). From 55e0a9115ee344955425d53d305ccf8fce33243a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 19:04:21 +0300 Subject: [PATCH 167/188] Update CHANGELOG.md Co-authored-by: Dan Roscigno --- CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 85363153e77..e0354f4ebb0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -65,7 +65,8 @@ * Allow readable size values (like `1TB`) in cache config. [#41688](https://github.com/ClickHouse/ClickHouse/pull/41688) ([Kseniia Sumarokova](https://github.com/kssenii)). * ClickHouse could cache stale DNS entries for some period of time (15 seconds by default) until the cache won't be updated asynchronously. During these periods ClickHouse can nevertheless try to establish a connection and produce errors. This behavior is fixed. [#41707](https://github.com/ClickHouse/ClickHouse/pull/41707) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Add interactive history search with fzf-like utility (fzf/sk) for `clickhouse-client`/`clickhouse-local` (note you can use `FZF_DEFAULT_OPTS`/`SKIM_DEFAULT_OPTIONS` to additionally configure the behavior). [#41730](https://github.com/ClickHouse/ClickHouse/pull/41730) ([Azat Khuzhin](https://github.com/azat)). -* For client when connecting to a secure server with invalid certificate only allow to proceed with '--accept-certificate' flag. [#41743](https://github.com/ClickHouse/ClickHouse/pull/41743) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* +Only allow clients connecting to a secure server with an invalid certificate only to proceed with the '--accept-certificate' flag. [#41743](https://github.com/ClickHouse/ClickHouse/pull/41743) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). * Add function `tryBase58Decode`, similar to the existing function `tryBase64Decode`. [#41824](https://github.com/ClickHouse/ClickHouse/pull/41824) ([Robert Schulze](https://github.com/rschu1ze)). * Improve feedback when replacing partition with different primary key. Fixes [#34798](https://github.com/ClickHouse/ClickHouse/issues/34798). [#41838](https://github.com/ClickHouse/ClickHouse/pull/41838) ([Salvatore](https://github.com/tbsal)). * Fix parallel parsing: segmentator now checks `max_block_size`. This fixed memory overallocation in case of parallel parsing and small LIMIT. [#41852](https://github.com/ClickHouse/ClickHouse/pull/41852) ([Vitaly Baranov](https://github.com/vitlibar)). From e28f9402c79f0d8085268ec4b77e709419dd5c2e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 19:04:31 +0300 Subject: [PATCH 168/188] Update CHANGELOG.md Co-authored-by: Dan Roscigno --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e0354f4ebb0..c9ec975df9b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -72,7 +72,7 @@ Only allow clients connecting to a secure server with an invalid certificate onl * Fix parallel parsing: segmentator now checks `max_block_size`. This fixed memory overallocation in case of parallel parsing and small LIMIT. [#41852](https://github.com/ClickHouse/ClickHouse/pull/41852) ([Vitaly Baranov](https://github.com/vitlibar)). * Don't add "TABLE_IS_DROPPED" exception to `system.errors` if it's happened during SELECT from a system table and was ignored. [#41908](https://github.com/ClickHouse/ClickHouse/pull/41908) ([AlfVII](https://github.com/AlfVII)). * Improve option `enable_extended_results_for_datetime_functions` to return results of type DateTime64 for functions `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute` and `timeSlot`. [#41910](https://github.com/ClickHouse/ClickHouse/pull/41910) ([Roman Vasin](https://github.com/rvasin)). -* Improve `DateTime` type inference for text formats. Now it respect setting `date_time_input_format` and doesn't try to infer datetimes from numbers as timestamps. Closes [#41389](https://github.com/ClickHouse/ClickHouse/issues/41389) Closes [#42206](https://github.com/ClickHouse/ClickHouse/issues/42206). [#41912](https://github.com/ClickHouse/ClickHouse/pull/41912) ([Kruglov Pavel](https://github.com/Avogar)). +* Improve `DateTime` type inference for text formats. Now it respects setting `date_time_input_format` and doesn't try to infer datetimes from numbers as timestamps. Closes [#41389](https://github.com/ClickHouse/ClickHouse/issues/41389) Closes [#42206](https://github.com/ClickHouse/ClickHouse/issues/42206). [#41912](https://github.com/ClickHouse/ClickHouse/pull/41912) ([Kruglov Pavel](https://github.com/Avogar)). * Remove confusing warning when inserting with `perform_ttl_move_on_insert` = false. [#41980](https://github.com/ClickHouse/ClickHouse/pull/41980) ([Vitaly Baranov](https://github.com/vitlibar)). * Allow user to write `countState(*)` similar to `count(*)`. This closes [#9338](https://github.com/ClickHouse/ClickHouse/issues/9338). [#41983](https://github.com/ClickHouse/ClickHouse/pull/41983) ([Amos Bird](https://github.com/amosbird)). * Fix `rankCorr` size overflow. [#42020](https://github.com/ClickHouse/ClickHouse/pull/42020) ([Duc Canh Le](https://github.com/canhld94)). From f6139c82b6bc166ffd8ee894c79ce37d19f9e39b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 19:04:37 +0300 Subject: [PATCH 169/188] Update CHANGELOG.md Co-authored-by: Dan Roscigno --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c9ec975df9b..ea74314ef57 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -79,7 +79,7 @@ Only allow clients connecting to a secure server with an invalid certificate onl * Added an option to specify an arbitrary string as an environment name in the Sentry's config for more handy reports. [#42037](https://github.com/ClickHouse/ClickHouse/pull/42037) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Fix parsing out-of-range Date from CSV. [#42044](https://github.com/ClickHouse/ClickHouse/pull/42044) ([Andrey Zvonov](https://github.com/zvonand)). * `parseDataTimeBestEffort` now supports comma between date and time. Closes [#42038](https://github.com/ClickHouse/ClickHouse/issues/42038). [#42049](https://github.com/ClickHouse/ClickHouse/pull/42049) ([flynn](https://github.com/ucasfl)). -* Improved stale replica recovery process for `ReplicatedMergeTree`. If a lost replica have some parts which absent on a healthy replica, but these parts should appear in future according to replication queue of the healthy replica, then lost replica will keep such parts instead of detaching them. [#42134](https://github.com/ClickHouse/ClickHouse/pull/42134) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Improved stale replica recovery process for `ReplicatedMergeTree`. If a lost replica has some parts which are absent from a healthy replica, but these parts should appear in the future according to the replication queue of the healthy replica, then the lost replica will keep such parts instead of detaching them. [#42134](https://github.com/ClickHouse/ClickHouse/pull/42134) ([Alexander Tokmakov](https://github.com/tavplubix)). * Add a possibility to use `Date32` arguments for date_diff function. Fix issue in date_diff function when using DateTime64 arguments with start date before Unix epoch and end date after Unix epoch. [#42308](https://github.com/ClickHouse/ClickHouse/pull/42308) ([Roman Vasin](https://github.com/rvasin)). * When uploading big parts to Minio, 'Complete Multipart Upload' can take a long time. Minio sends heartbeats every 10 seconds (see https://github.com/minio/minio/pull/7198). But clickhouse times out earlier, because the default send/receive timeout is [set](https://github.com/ClickHouse/ClickHouse/blob/cc24fcd6d5dfb67f5f66f5483e986bd1010ad9cf/src/IO/S3/PocoHTTPClient.cpp#L123) to 5 seconds. [#42321](https://github.com/ClickHouse/ClickHouse/pull/42321) ([filimonov](https://github.com/filimonov)). * Fix rarely invalid cast of aggregate state types with complex types such as Decimal. This fixes [#42408](https://github.com/ClickHouse/ClickHouse/issues/42408). [#42417](https://github.com/ClickHouse/ClickHouse/pull/42417) ([Amos Bird](https://github.com/amosbird)). From aa8434610787b8e9a2d164499be8f20843f24002 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 19:04:43 +0300 Subject: [PATCH 170/188] Update CHANGELOG.md Co-authored-by: Dan Roscigno --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ea74314ef57..4a1c8ed9c67 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -80,7 +80,7 @@ Only allow clients connecting to a secure server with an invalid certificate onl * Fix parsing out-of-range Date from CSV. [#42044](https://github.com/ClickHouse/ClickHouse/pull/42044) ([Andrey Zvonov](https://github.com/zvonand)). * `parseDataTimeBestEffort` now supports comma between date and time. Closes [#42038](https://github.com/ClickHouse/ClickHouse/issues/42038). [#42049](https://github.com/ClickHouse/ClickHouse/pull/42049) ([flynn](https://github.com/ucasfl)). * Improved stale replica recovery process for `ReplicatedMergeTree`. If a lost replica has some parts which are absent from a healthy replica, but these parts should appear in the future according to the replication queue of the healthy replica, then the lost replica will keep such parts instead of detaching them. [#42134](https://github.com/ClickHouse/ClickHouse/pull/42134) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Add a possibility to use `Date32` arguments for date_diff function. Fix issue in date_diff function when using DateTime64 arguments with start date before Unix epoch and end date after Unix epoch. [#42308](https://github.com/ClickHouse/ClickHouse/pull/42308) ([Roman Vasin](https://github.com/rvasin)). +* Add a possibility to use `Date32` arguments for date_diff function. Fix issue in date_diff function when using DateTime64 arguments with a start date before Unix epoch and end date after Unix epoch. [#42308](https://github.com/ClickHouse/ClickHouse/pull/42308) ([Roman Vasin](https://github.com/rvasin)). * When uploading big parts to Minio, 'Complete Multipart Upload' can take a long time. Minio sends heartbeats every 10 seconds (see https://github.com/minio/minio/pull/7198). But clickhouse times out earlier, because the default send/receive timeout is [set](https://github.com/ClickHouse/ClickHouse/blob/cc24fcd6d5dfb67f5f66f5483e986bd1010ad9cf/src/IO/S3/PocoHTTPClient.cpp#L123) to 5 seconds. [#42321](https://github.com/ClickHouse/ClickHouse/pull/42321) ([filimonov](https://github.com/filimonov)). * Fix rarely invalid cast of aggregate state types with complex types such as Decimal. This fixes [#42408](https://github.com/ClickHouse/ClickHouse/issues/42408). [#42417](https://github.com/ClickHouse/ClickHouse/pull/42417) ([Amos Bird](https://github.com/amosbird)). * Allow to use `Date32` arguments for `dateName` function. [#42554](https://github.com/ClickHouse/ClickHouse/pull/42554) ([Roman Vasin](https://github.com/rvasin)). From dd4ff1e0e5978e3ccb7feecbd4890efbdcb013f5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 19:04:51 +0300 Subject: [PATCH 171/188] Update CHANGELOG.md Co-authored-by: Dan Roscigno --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4a1c8ed9c67..131804043e2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -100,7 +100,7 @@ Only allow clients connecting to a secure server with an invalid certificate onl * Choose correct aggregation method for `LowCardinality` with big integer types. [#42342](https://github.com/ClickHouse/ClickHouse/pull/42342) ([Duc Canh Le](https://github.com/canhld94)). * Several fixes for `web` disk. [#41652](https://github.com/ClickHouse/ClickHouse/pull/41652) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fixes issue when docker run will fail if `https_port` is not present in config. [#41693](https://github.com/ClickHouse/ClickHouse/pull/41693) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fixes an issue that causes docker run to fail if `https_port` is not present in config. [#41693](https://github.com/ClickHouse/ClickHouse/pull/41693) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). * Mutations were not cancelled properly on server shutdown or `SYSTEM STOP MERGES` query and cancellation might take long time, it's fixed. [#41699](https://github.com/ClickHouse/ClickHouse/pull/41699) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix wrong result of queries with `ORDER BY` or `GROUP BY` by columns from prefix of sorting key, wrapped into monotonic functions, with enable "read in order" optimization (settings `optimize_read_in_order` and `optimize_aggregation_in_order`). [#41701](https://github.com/ClickHouse/ClickHouse/pull/41701) ([Anton Popov](https://github.com/CurtizJ)). * Fix possible crash in `SELECT` from `Merge` table with enabled `optimize_monotonous_functions_in_order_by` setting. Fixes [#41269](https://github.com/ClickHouse/ClickHouse/issues/41269). [#41740](https://github.com/ClickHouse/ClickHouse/pull/41740) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). From 67212ad575feb4930b3082ada7a627f439a5b9b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 19:04:59 +0300 Subject: [PATCH 172/188] Update CHANGELOG.md Co-authored-by: Dan Roscigno --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 131804043e2..c2def2f787e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -110,7 +110,7 @@ Only allow clients connecting to a secure server with an invalid certificate onl * 22.9 might fail to startup `ReplicatedMergeTree` table if that table was created by 20.3 or older version and was never altered, it's fixed. Fixes [#41742](https://github.com/ClickHouse/ClickHouse/issues/41742). [#41796](https://github.com/ClickHouse/ClickHouse/pull/41796) ([Alexander Tokmakov](https://github.com/tavplubix)). * When the batch sending fails for some reason, it cannot be automatically recovered, and if it is not processed in time, it will lead to accumulation, and the printed error message will become longer and longer, which will cause the http thread to block. [#41813](https://github.com/ClickHouse/ClickHouse/pull/41813) ([zhongyuankai](https://github.com/zhongyuankai)). * Fix compact parts with compressed marks setting. Fixes [#41783](https://github.com/ClickHouse/ClickHouse/issues/41783) and [#41746](https://github.com/ClickHouse/ClickHouse/issues/41746). [#41823](https://github.com/ClickHouse/ClickHouse/pull/41823) ([alesapin](https://github.com/alesapin)). -* Old versions of Replicated database doesn't have a special marker in [Zoo]Keeper. We need to check only whether the node contains come obscure data instead of special mark. [#41875](https://github.com/ClickHouse/ClickHouse/pull/41875) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Old versions of Replicated database don't have a special marker in [Zoo]Keeper. We need to check only whether the node contains come obscure data instead of special mark. [#41875](https://github.com/ClickHouse/ClickHouse/pull/41875) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Fix possible exception in fs cache. [#41884](https://github.com/ClickHouse/ClickHouse/pull/41884) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix `use_environment_credentials` for s3 table function. [#41970](https://github.com/ClickHouse/ClickHouse/pull/41970) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fixed "Directory already exists and is not empty" error on detaching broken part that might prevent `ReplicatedMergeTree` table from starting replication. Fixes [#40957](https://github.com/ClickHouse/ClickHouse/issues/40957). [#41981](https://github.com/ClickHouse/ClickHouse/pull/41981) ([Alexander Tokmakov](https://github.com/tavplubix)). From 655233e1c5a424480f271926f08a69359ad5b7c8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Oct 2022 09:13:02 +0000 Subject: [PATCH 173/188] Add convenience typedefs for Date/Date32/DateTime/DateTime64 columns --- src/Columns/ColumnsDateTime.h | 20 +++++++++++++++++ src/Functions/FunctionsTimeWindow.cpp | 13 ++++++----- src/Functions/dateDiff.cpp | 32 +++++++++++++-------------- src/Functions/makeDate.cpp | 9 ++++---- src/Functions/nowInBlock.cpp | 4 ++-- src/Functions/timeSlots.cpp | 13 ++++++----- src/Functions/toStartOfInterval.cpp | 9 ++++---- 7 files changed, 62 insertions(+), 38 deletions(-) create mode 100644 src/Columns/ColumnsDateTime.h diff --git a/src/Columns/ColumnsDateTime.h b/src/Columns/ColumnsDateTime.h new file mode 100644 index 00000000000..90d21ed5ff7 --- /dev/null +++ b/src/Columns/ColumnsDateTime.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** Convenience typedefs for columns of SQL types Date, Date32, DateTime and DateTime64. */ + +using ColumnDate = DataTypeDate::ColumnType; +using ColumnDate32 = DataTypeDate32::ColumnType; +using ColumnDateTime = DataTypeDateTime::ColumnType; +using ColumnDateTime64 = DataTypeDateTime64::ColumnType; + +} diff --git a/src/Functions/FunctionsTimeWindow.cpp b/src/Functions/FunctionsTimeWindow.cpp index 286ed4a729d..a47fc71c335 100644 --- a/src/Functions/FunctionsTimeWindow.cpp +++ b/src/Functions/FunctionsTimeWindow.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -157,7 +158,7 @@ struct TimeWindowImpl const auto & interval_column = arguments[1]; const auto & from_datatype = *time_column.type.get(); const auto which_type = WhichDataType(from_datatype); - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); if (!which_type.isDateTime() || !time_column_vec) throw Exception( @@ -198,7 +199,7 @@ struct TimeWindowImpl } template - static ColumnPtr executeTumble(const ColumnUInt32 & time_column, UInt64 num_units, const DateLUTImpl & time_zone) + static ColumnPtr executeTumble(const ColumnDateTime & time_column, UInt64 num_units, const DateLUTImpl & time_zone) { const auto & time_data = time_column.getData(); size_t size = time_column.size(); @@ -342,7 +343,7 @@ struct TimeWindowImpl const auto & hop_interval_column = arguments[1]; const auto & window_interval_column = arguments[2]; const auto & from_datatype = *time_column.type.get(); - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) throw Exception( @@ -402,7 +403,7 @@ struct TimeWindowImpl template static ColumnPtr - executeHop(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) + executeHop(const ColumnDateTime & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) { const auto & time_data = time_column.getData(); size_t size = time_column.size(); @@ -491,7 +492,7 @@ struct TimeWindowImpl const auto & hop_interval_column = arguments[1]; const auto & window_interval_column = arguments[2]; const auto & from_datatype = *time_column.type.get(); - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) throw Exception( @@ -551,7 +552,7 @@ struct TimeWindowImpl template static ColumnPtr - executeHopSlice(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) + executeHopSlice(const ColumnDateTime & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) { Int64 gcd_num_units = std::gcd(hop_num_units, window_num_units); diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index b33fcf32de1..f5a4b50fb54 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -44,7 +45,6 @@ namespace */ class FunctionDateDiff : public IFunction { - using ColumnDateTime64 = ColumnDecimal; public: static constexpr auto name = "dateDiff"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } @@ -141,19 +141,19 @@ private: const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, ColumnInt64::Container & result) const { - if (const auto * x_vec_16 = checkAndGetColumn(&x)) + if (const auto * x_vec_16 = checkAndGetColumn(&x)) dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_32 = checkAndGetColumn(&x)) + else if (const auto * x_vec_32 = checkAndGetColumn(&x)) dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_32_s = checkAndGetColumn(&x)) + else if (const auto * x_vec_32_s = checkAndGetColumn(&x)) dispatchForSecondColumn(*x_vec_32_s, y, timezone_x, timezone_y, result); else if (const auto * x_vec_64 = checkAndGetColumn(&x)) dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); - else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) + else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) + else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_32_s = checkAndGetColumnConst(&x)) + else if (const auto * x_const_32_s = checkAndGetColumnConst(&x)) dispatchConstForSecondColumn(x_const_32_s->getValue(), y, timezone_x, timezone_y, result); else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); @@ -169,19 +169,19 @@ private: const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, ColumnInt64::Container & result) const { - if (const auto * y_vec_16 = checkAndGetColumn(&y)) + if (const auto * y_vec_16 = checkAndGetColumn(&y)) vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) + else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) vectorVector(x, *y_vec_32_s, timezone_x, timezone_y, result); else if (const auto * y_vec_64 = checkAndGetColumn(&y)) vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); - else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) + else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) + else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_32_s = checkAndGetColumnConst(&y)) + else if (const auto * y_const_32_s = checkAndGetColumnConst(&y)) vectorConstant(x, y_const_32_s->getValue(), timezone_x, timezone_y, result); else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); @@ -197,11 +197,11 @@ private: const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, ColumnInt64::Container & result) const { - if (const auto * y_vec_16 = checkAndGetColumn(&y)) + if (const auto * y_vec_16 = checkAndGetColumn(&y)) constantVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) constantVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) + else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) constantVector(x, *y_vec_32_s, timezone_x, timezone_y, result); else if (const auto * y_vec_64 = checkAndGetColumn(&y)) constantVector(x, *y_vec_64, timezone_x, timezone_y, result); diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index c9571a7333d..7ebca71af13 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -149,7 +150,7 @@ struct MakeDateTraits { static constexpr auto name = "makeDate"; using ReturnDataType = DataTypeDate; - using ReturnColumnType = ColumnUInt16; + using ReturnColumnType = ColumnDate; static constexpr auto MIN_YEAR = 1970; static constexpr auto MAX_YEAR = 2149; @@ -162,7 +163,7 @@ struct MakeDate32Traits { static constexpr auto name = "makeDate32"; using ReturnDataType = DataTypeDate32; - using ReturnColumnType = ColumnInt32; + using ReturnColumnType = ColumnDate32; static constexpr auto MIN_YEAR = 1900; static constexpr auto MAX_YEAR = 2299; @@ -267,7 +268,7 @@ public: Columns converted_arguments; convertRequiredArguments(arguments, converted_arguments); - auto res_column = ColumnUInt32::create(input_rows_count); + auto res_column = ColumnDateTime::create(input_rows_count); auto & result_data = res_column->getData(); const auto & year_data = typeid_cast(*converted_arguments[0]).getData(); @@ -365,7 +366,7 @@ public: fraction_data = &typeid_cast(*converted_arguments[6]).getData(); } - auto res_column = ColumnDecimal::create(input_rows_count, static_cast(precision)); + auto res_column = ColumnDateTime64::create(input_rows_count, static_cast(precision)); auto & result_data = res_column->getData(); const auto & year_data = typeid_cast(*converted_arguments[0]).getData(); diff --git a/src/Functions/nowInBlock.cpp b/src/Functions/nowInBlock.cpp index c771d83225a..b1764590fda 100644 --- a/src/Functions/nowInBlock.cpp +++ b/src/Functions/nowInBlock.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include namespace DB @@ -74,7 +74,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override { - return ColumnUInt32::create(input_rows_count, static_cast(time(nullptr))); + return ColumnDateTime::create(input_rows_count, static_cast(time(nullptr))); } }; diff --git a/src/Functions/timeSlots.cpp b/src/Functions/timeSlots.cpp index e986e32d76f..72d6059e0a1 100644 --- a/src/Functions/timeSlots.cpp +++ b/src/Functions/timeSlots.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -300,11 +301,11 @@ public: throw Exception("Third argument for function " + getName() + " must be greater than zero", ErrorCodes::ILLEGAL_COLUMN); } - const auto * dt_starts = checkAndGetColumn(arguments[0].column.get()); - const auto * dt_const_starts = checkAndGetColumnConst(arguments[0].column.get()); + const auto * dt_starts = checkAndGetColumn(arguments[0].column.get()); + const auto * dt_const_starts = checkAndGetColumnConst(arguments[0].column.get()); - const auto * durations = checkAndGetColumn(arguments[1].column.get()); - const auto * const_durations = checkAndGetColumnConst(arguments[1].column.get()); + const auto * durations = checkAndGetColumn(arguments[1].column.get()); + const auto * const_durations = checkAndGetColumnConst(arguments[1].column.get()); auto res = ColumnArray::create(ColumnUInt32::create()); ColumnUInt32::Container & res_values = typeid_cast(res->getData()).getData(); @@ -341,8 +342,8 @@ public: time_slot_scale = assert_cast(arguments[2].type.get())->getScale(); } - const auto * starts = checkAndGetColumn(arguments[0].column.get()); - const auto * const_starts = checkAndGetColumnConst(arguments[0].column.get()); + const auto * starts = checkAndGetColumn(arguments[0].column.get()); + const auto * const_starts = checkAndGetColumnConst(arguments[0].column.get()); const auto * durations = checkAndGetColumn>(arguments[1].column.get()); const auto * const_durations = checkAndGetColumnConst>(arguments[1].column.get()); diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 32fe574f56a..3054cf280d9 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -437,7 +438,7 @@ private: if (which_type.isDateTime64()) { - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); auto scale = assert_cast(from_datatype).getScale(); if (time_column_vec) @@ -445,19 +446,19 @@ private: } if (which_type.isDateTime()) { - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); } if (which_type.isDate()) { - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); } if (which_type.isDate32()) { - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); if (time_column_vec) return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); } From b58ca49f0560f72653ac9cda38fafefa59670e66 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Oct 2022 17:26:14 +0000 Subject: [PATCH 174/188] fix bad cast in arrayElement with Map and Nullable index --- src/Functions/array/arrayElement.cpp | 6 +++++- .../02473_map_element_nullable.reference | 16 ++++++++++++++++ .../02473_map_element_nullable.sql | 19 +++++++++++++++++++ 3 files changed, 40 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02473_map_element_nullable.reference create mode 100644 tests/queries/0_stateless/02473_map_element_nullable.sql diff --git a/src/Functions/array/arrayElement.cpp b/src/Functions/array/arrayElement.cpp index acd8f89ffe5..59224096d3c 100644 --- a/src/Functions/array/arrayElement.cpp +++ b/src/Functions/array/arrayElement.cpp @@ -1025,12 +1025,14 @@ ColumnPtr FunctionArrayElement::executeMap( if (col_const_map) values_array = ColumnConst::create(values_array, input_rows_count); + const auto & type_map = assert_cast(*arguments[0].type); + /// Prepare arguments to call arrayElement for array with values and calculated indices at previous step. ColumnsWithTypeAndName new_arguments = { { values_array, - std::make_shared(result_type), + std::make_shared(type_map.getValueType()), "" }, { @@ -1086,7 +1088,9 @@ ColumnPtr FunctionArrayElement::executeImpl(const ColumnsWithTypeAndName & argum col_array = checkAndGetColumn(arguments[0].column.get()); if (col_array) + { is_array_of_nullable = isColumnNullable(col_array->getData()); + } else { col_const_array = checkAndGetColumnConstData(arguments[0].column.get()); diff --git a/tests/queries/0_stateless/02473_map_element_nullable.reference b/tests/queries/0_stateless/02473_map_element_nullable.reference new file mode 100644 index 00000000000..84a9ba03bb4 --- /dev/null +++ b/tests/queries/0_stateless/02473_map_element_nullable.reference @@ -0,0 +1,16 @@ +2 \N \N +2 \N \N +2 \N \N +2 \N \N +2 \N \N +2 \N \N +2 \N \N +2 \N \N +2 \N \N +2 \N \N +2 \N \N +2 \N \N +2 \N \N +2 \N \N +2 \N \N +2 \N \N diff --git a/tests/queries/0_stateless/02473_map_element_nullable.sql b/tests/queries/0_stateless/02473_map_element_nullable.sql new file mode 100644 index 00000000000..e9c351d112c --- /dev/null +++ b/tests/queries/0_stateless/02473_map_element_nullable.sql @@ -0,0 +1,19 @@ +WITH map(1, 2, 3, NULL) AS m SELECT m[toNullable(1)], m[toNullable(2)], m[toNullable(3)]; +WITH map(1, 2, 3, NULL) AS m SELECT m[materialize(toNullable(1))], m[materialize(toNullable(2))], m[materialize(toNullable(3))]; +WITH materialize(map(1, 2, 3, NULL)) AS m SELECT m[toNullable(1)], m[toNullable(2)], m[toNullable(3)]; +WITH materialize(map(1, 2, 3, NULL)) AS m SELECT m[materialize(toNullable(1))], m[materialize(toNullable(2))], m[materialize(toNullable(3))]; + +WITH map('a', 2, 'b', NULL) AS m SELECT m[toNullable('a')], m[toNullable('b')], m[toNullable('c')]; +WITH map('a', 2, 'b', NULL) AS m SELECT m[materialize(toNullable('a'))], m[materialize(toNullable('b'))], m[materialize(toNullable('c'))]; +WITH materialize(map('a', 2, 'b', NULL)) AS m SELECT m[toNullable('a')], m[toNullable('b')], m[toNullable('c')]; +WITH materialize(map('a', 2, 'b', NULL)) AS m SELECT m[materialize(toNullable('a'))], m[materialize(toNullable('b'))], m[materialize(toNullable('c'))]; + +WITH map(1, 2, 3, NULL) AS m SELECT m[1], m[2], m[3]; +WITH map(1, 2, 3, NULL) AS m SELECT m[materialize(1)], m[materialize(2)], m[materialize(3)]; +WITH materialize(map(1, 2, 3, NULL)) AS m SELECT m[1], m[2], m[3]; +WITH materialize(map(1, 2, 3, NULL)) AS m SELECT m[materialize(1)], m[materialize(2)], m[materialize(3)]; + +WITH map('a', 2, 'b', NULL) AS m SELECT m['a'], m['b'], m['c']; +WITH map('a', 2, 'b', NULL) AS m SELECT m[materialize('a')], m[materialize('b')], m[materialize('c')]; +WITH materialize(map('a', 2, 'b', NULL)) AS m SELECT m['a'], m['b'], m['c']; +WITH materialize(map('a', 2, 'b', NULL)) AS m SELECT m[materialize('a')], m[materialize('b')], m[materialize('c')]; From 787c25243798d3a23a5779b3cf82d1ce4a4c84b9 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 24 Oct 2022 21:07:52 +0200 Subject: [PATCH 175/188] Add a template for installation issues --- .../ISSUE_TEMPLATE/96_installation-issues.md | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) create mode 100644 .github/ISSUE_TEMPLATE/96_installation-issues.md diff --git a/.github/ISSUE_TEMPLATE/96_installation-issues.md b/.github/ISSUE_TEMPLATE/96_installation-issues.md new file mode 100644 index 00000000000..c322ccc92ce --- /dev/null +++ b/.github/ISSUE_TEMPLATE/96_installation-issues.md @@ -0,0 +1,29 @@ +--- +name: Installation issue +about: Issue with ClickHouse installation from https://clickhouse.com/docs/en/install/ +title: '' +labels: comp-install +assignees: '' + +--- + +**Installation type** + +Packages, docker, single binary, curl? + +**Source of the ClickHouse** + +A link to the source. Or the command you've tried + +**Expected result** + +What you expected + +**The actual result** + +What you get + +**How to reproduce** + +* For Linux-based operating systems: provide a script for clear docker container from the official image +* For anything else: steps to reproduce on as much as possible clear system From 171ed8dd4e1e050f34a19b3702381a112a00e159 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 24 Oct 2022 19:52:51 +0000 Subject: [PATCH 176/188] Apply suggestions from review --- src/Parsers/ExpressionListParsers.cpp | 32 +++++++++++++-------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index a38d7ab0c98..6336b92de1b 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -1096,7 +1096,7 @@ public: class CastLayer : public Layer { public: - CastLayer() : Layer(true, true) {} + CastLayer() : Layer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { @@ -1196,7 +1196,7 @@ public: class ExtractLayer : public BaseLayer { public: - ExtractLayer() : BaseLayer(true, true) {} + ExtractLayer() : BaseLayer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} bool getResult(ASTPtr & node) override { @@ -1267,7 +1267,7 @@ private: class SubstringLayer : public Layer { public: - SubstringLayer() : Layer(true, true) {} + SubstringLayer() : Layer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} bool getResult(ASTPtr & node) override { @@ -1329,7 +1329,7 @@ public: class PositionLayer : public Layer { public: - PositionLayer() : Layer(true, true) {} + PositionLayer() : Layer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} bool getResult(ASTPtr & node) override { @@ -1399,7 +1399,7 @@ public: class ExistsLayer : public Layer { public: - ExistsLayer() : Layer(true, true) {} + ExistsLayer() : Layer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} bool parse(IParser::Pos & pos, Expected & expected, Action & /*action*/) override { @@ -1426,7 +1426,7 @@ class TrimLayer : public Layer { public: TrimLayer(bool trim_left_, bool trim_right_) - : Layer(true, true), trim_left(trim_left_), trim_right(trim_right_) {} + : Layer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true), trim_left(trim_left_), trim_right(trim_right_) {} bool getResult(ASTPtr & node) override { @@ -1588,7 +1588,7 @@ class DateAddLayer : public BaseLayer { public: - DateDiffLayer() : BaseLayer(true, true) {} + DateDiffLayer() : BaseLayer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} bool getResult(ASTPtr & node) override { @@ -1696,7 +1696,7 @@ private: class IntervalLayer : public Layer { public: - IntervalLayer() : Layer(true, true) {} + IntervalLayer() : Layer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { @@ -1775,7 +1775,7 @@ private: class CaseLayer : public Layer { public: - CaseLayer() : Layer(true, true) {} + CaseLayer() : Layer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { @@ -2444,13 +2444,13 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po { ParserAlias alias_parser(layers.back()->allow_alias_without_as_keyword); auto old_pos = pos; - if (!layers.back()->parsed_alias && layers.back()->allow_alias && alias_parser.parse(pos, tmp, expected)) + if (layers.back()->allow_alias && + !layers.back()->parsed_alias && + alias_parser.parse(pos, tmp, expected) && + layers.back()->insertAlias(tmp)) { - if (layers.back()->insertAlias(tmp)) - { - layers.back()->parsed_alias = true; - return Action::OPERATOR; - } + layers.back()->parsed_alias = true; + return Action::OPERATOR; } pos = old_pos; return Action::NONE; From 8fcbef938b496de31014669cac6cad8c075ca23d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 24 Oct 2022 23:01:06 +0200 Subject: [PATCH 177/188] Fix typo in cmake code related to fuzzing (#42627) --- cmake/sanitize.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 73610545009..f0cef54b0b8 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -85,7 +85,7 @@ if (SANITIZE) # and they have a bunch of flags not halt the program if UIO happend and even to silence that warnings. # But for unknown reason that flags don't work with ClickHouse or we don't understand how to properly use them, # that's why we often receive reports about UIO. The simplest way to avoid this is just set this flag here. - set(UBSAN_FLAGS "${SAN_FLAGS} -fno-sanitize=unsigned-integer-overflow") + set(UBSAN_FLAGS "${UBSAN_FLAGS} -fno-sanitize=unsigned-integer-overflow") endif() if (COMPILER_CLANG) set (UBSAN_FLAGS "${UBSAN_FLAGS} -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/ubsan_suppressions.txt") From 4e294b9bfc11d4347989e6d693353787b4dd2078 Mon Sep 17 00:00:00 2001 From: Guo Wangyang Date: Tue, 25 Oct 2022 05:11:46 +0800 Subject: [PATCH 178/188] ColumnVector: re-enable AVX512_VBMI/AVX512_VBMI2 optimized filter and index (#41765) * Revert "Merge pull request #41752 from ClickHouse/revert-revert-revert" This reverts commit 7bb245720abfd0550b6a2777a4aab654c0a077c6, reversing changes made to 78002dc2482492a5d65b8a55adb97718522ff64e. * Revert "Merge pull request #41797 from ClickHouse/revert-41247" This reverts commit 69b13d9f31603508138f1a68f84ba0c567270e90, reversing changes made to 1719e000ea4bd3f461f06c3e647a3e7f3fc35a4d. * AVX512VBMI::vectorIndexImpl: Fix heap buffer overflow when limit == 0 * use SIMD_ELEMENTS instead of SIMD_BYTES to avoid confusing --- src/Columns/ColumnVector.cpp | 166 ++++++++++++++++++---- src/Columns/ColumnVector.h | 139 +++++++++++++++++- src/Columns/tests/gtest_column_vector.cpp | 159 +++++++++++++++++++++ src/Common/CpuId.h | 6 + src/Common/TargetSpecific.cpp | 7 +- src/Common/TargetSpecific.h | 33 ++++- 6 files changed, 473 insertions(+), 37 deletions(-) create mode 100644 src/Columns/tests/gtest_column_vector.cpp diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 19a6e6bfa87..30e7423fde0 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -12,12 +12,14 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include @@ -25,6 +27,10 @@ # include #endif +#if USE_MULTITARGET_CODE +# include +#endif + #if USE_EMBEDDED_COMPILER #include #include @@ -471,6 +477,128 @@ void ColumnVector::insertRangeFrom(const IColumn & src, size_t start, size_t memcpy(data.data() + old_size, &src_vec.data[start], length * sizeof(data[0])); } +static inline UInt64 blsr(UInt64 mask) +{ +#ifdef __BMI__ + return _blsr_u64(mask); +#else + return mask & (mask-1); +#endif +} + +DECLARE_DEFAULT_CODE( +template +inline void doFilterAligned(const UInt8 *& filt_pos, const UInt8 *& filt_end_aligned, const T *& data_pos, Container & res_data) +{ + while (filt_pos < filt_end_aligned) + { + UInt64 mask = bytes64MaskToBits64Mask(filt_pos); + + if (0xffffffffffffffff == mask) + { + res_data.insert(data_pos, data_pos + SIMD_ELEMENTS); + } + else + { + while (mask) + { + size_t index = std::countr_zero(mask); + res_data.push_back(data_pos[index]); + mask = blsr(mask); + } + } + + filt_pos += SIMD_ELEMENTS; + data_pos += SIMD_ELEMENTS; + } +} +) + +namespace +{ +template +void resize(Container & res_data, size_t reserve_size) +{ +#if defined(MEMORY_SANITIZER) + res_data.resize_fill(reserve_size, static_cast(0)); // MSan doesn't recognize that all allocated memory is written by AVX-512 intrinsics. +#else + res_data.resize(reserve_size); +#endif +} +} + +DECLARE_AVX512VBMI2_SPECIFIC_CODE( +template +inline void compressStoreAVX512(const void *src, void *dst, const UInt64 mask) +{ + __m512i vsrc = _mm512_loadu_si512(src); + if constexpr (ELEMENT_WIDTH == 1) + _mm512_mask_compressstoreu_epi8(dst, static_cast<__mmask64>(mask), vsrc); + else if constexpr (ELEMENT_WIDTH == 2) + _mm512_mask_compressstoreu_epi16(dst, static_cast<__mmask32>(mask), vsrc); + else if constexpr (ELEMENT_WIDTH == 4) + _mm512_mask_compressstoreu_epi32(dst, static_cast<__mmask16>(mask), vsrc); + else if constexpr (ELEMENT_WIDTH == 8) + _mm512_mask_compressstoreu_epi64(dst, static_cast<__mmask8>(mask), vsrc); +} + +template +inline void doFilterAligned(const UInt8 *& filt_pos, const UInt8 *& filt_end_aligned, const T *& data_pos, Container & res_data) +{ + static constexpr size_t VEC_LEN = 64; /// AVX512 vector length - 64 bytes + static constexpr size_t ELEMENT_WIDTH = sizeof(T); + static constexpr size_t ELEMENTS_PER_VEC = VEC_LEN / ELEMENT_WIDTH; + static constexpr UInt64 KMASK = 0xffffffffffffffff >> (64 - ELEMENTS_PER_VEC); + + size_t current_offset = res_data.size(); + size_t reserve_size = res_data.size(); + size_t alloc_size = SIMD_ELEMENTS * 2; + + while (filt_pos < filt_end_aligned) + { + /// to avoid calling resize too frequently, resize to reserve buffer. + if (reserve_size - current_offset < SIMD_ELEMENTS) + { + reserve_size += alloc_size; + resize(res_data, reserve_size); + alloc_size *= 2; + } + + UInt64 mask = bytes64MaskToBits64Mask(filt_pos); + + if (0xffffffffffffffff == mask) + { + for (size_t i = 0; i < SIMD_ELEMENTS; i += ELEMENTS_PER_VEC) + _mm512_storeu_si512(reinterpret_cast(&res_data[current_offset + i]), + _mm512_loadu_si512(reinterpret_cast(data_pos + i))); + current_offset += SIMD_ELEMENTS; + } + else + { + if (mask) + { + for (size_t i = 0; i < SIMD_ELEMENTS; i += ELEMENTS_PER_VEC) + { + compressStoreAVX512(reinterpret_cast(data_pos + i), + reinterpret_cast(&res_data[current_offset]), mask & KMASK); + current_offset += std::popcount(mask & KMASK); + /// prepare mask for next iter, if ELEMENTS_PER_VEC = 64, no next iter + if (ELEMENTS_PER_VEC < 64) + { + mask >>= ELEMENTS_PER_VEC; + } + } + } + } + + filt_pos += SIMD_ELEMENTS; + data_pos += SIMD_ELEMENTS; + } + /// resize to the real size. + res_data.resize(current_offset); +} +) + template ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_size_hint) const { @@ -491,36 +619,18 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s /** A slightly more optimized version. * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. - * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. + * Therefore, we will optimistically check the parts of `SIMD_ELEMENTS` values. */ - static constexpr size_t SIMD_BYTES = 64; - const UInt8 * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + static constexpr size_t SIMD_ELEMENTS = 64; + const UInt8 * filt_end_aligned = filt_pos + size / SIMD_ELEMENTS * SIMD_ELEMENTS; - while (filt_pos < filt_end_aligned) - { - UInt64 mask = bytes64MaskToBits64Mask(filt_pos); - - if (0xffffffffffffffff == mask) - { - res_data.insert(data_pos, data_pos + SIMD_BYTES); - } - else - { - while (mask) - { - size_t index = std::countr_zero(mask); - res_data.push_back(data_pos[index]); - #ifdef __BMI__ - mask = _blsr_u64(mask); - #else - mask = mask & (mask-1); - #endif - } - } - - filt_pos += SIMD_BYTES; - data_pos += SIMD_BYTES; - } +#if USE_MULTITARGET_CODE + static constexpr bool VBMI2_CAPABLE = sizeof(T) == 1 || sizeof(T) == 2 || sizeof(T) == 4 || sizeof(T) == 8; + if (VBMI2_CAPABLE && isArchSupported(TargetArch::AVX512VBMI2)) + TargetSpecific::AVX512VBMI2::doFilterAligned(filt_pos, filt_end_aligned, data_pos, res_data); + else +#endif + TargetSpecific::Default::doFilterAligned(filt_pos, filt_end_aligned, data_pos, res_data); while (filt_pos < filt_end) { diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 0f388ef8ac3..a601dd8b405 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -7,11 +7,15 @@ #include #include #include +#include #include #include #include "config.h" +#if USE_MULTITARGET_CODE +# include +#endif namespace DB { @@ -391,6 +395,127 @@ protected: Container data; }; +DECLARE_DEFAULT_CODE( +template +inline void vectorIndexImpl(const Container & data, const PaddedPODArray & indexes, size_t limit, Container & res_data) +{ + for (size_t i = 0; i < limit; ++i) + res_data[i] = data[indexes[i]]; +} +); + +DECLARE_AVX512VBMI_SPECIFIC_CODE( +template +inline void vectorIndexImpl(const Container & data, const PaddedPODArray & indexes, size_t limit, Container & res_data) +{ + static constexpr UInt64 MASK64 = 0xffffffffffffffff; + const size_t limit64 = limit & ~63; + size_t pos = 0; + size_t data_size = data.size(); + + auto data_pos = reinterpret_cast(data.data()); + auto indexes_pos = reinterpret_cast(indexes.data()); + auto res_pos = reinterpret_cast(res_data.data()); + + if (limit == 0) + return; /// nothing to do, just return + + if (data_size <= 64) + { + /// one single mask load for table size <= 64 + __mmask64 last_mask = MASK64 >> (64 - data_size); + __m512i table1 = _mm512_maskz_loadu_epi8(last_mask, data_pos); + + /// 64 bytes table lookup using one single permutexvar_epi8 + while (pos < limit64) + { + __m512i vidx = _mm512_loadu_epi8(indexes_pos + pos); + __m512i out = _mm512_permutexvar_epi8(vidx, table1); + _mm512_storeu_epi8(res_pos + pos, out); + pos += 64; + } + /// tail handling + if (limit > limit64) + { + __mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit); + __m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos); + __m512i out = _mm512_permutexvar_epi8(vidx, table1); + _mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out); + } + } + else if (data_size <= 128) + { + /// table size (64, 128] requires 2 zmm load + __mmask64 last_mask = MASK64 >> (128 - data_size); + __m512i table1 = _mm512_loadu_epi8(data_pos); + __m512i table2 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 64); + + /// 128 bytes table lookup using one single permute2xvar_epi8 + while (pos < limit64) + { + __m512i vidx = _mm512_loadu_epi8(indexes_pos + pos); + __m512i out = _mm512_permutex2var_epi8(table1, vidx, table2); + _mm512_storeu_epi8(res_pos + pos, out); + pos += 64; + } + if (limit > limit64) + { + __mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit); + __m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos); + __m512i out = _mm512_permutex2var_epi8(table1, vidx, table2); + _mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out); + } + } + else + { + if (data_size > 256) + { + /// byte index will not exceed 256 boundary. + data_size = 256; + } + + __m512i table1 = _mm512_loadu_epi8(data_pos); + __m512i table2 = _mm512_loadu_epi8(data_pos + 64); + __m512i table3, table4; + if (data_size <= 192) + { + /// only 3 tables need to load if size <= 192 + __mmask64 last_mask = MASK64 >> (192 - data_size); + table3 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 128); + table4 = _mm512_setzero_si512(); + } + else + { + __mmask64 last_mask = MASK64 >> (256 - data_size); + table3 = _mm512_loadu_epi8(data_pos + 128); + table4 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 192); + } + + /// 256 bytes table lookup can use: 2 permute2xvar_epi8 plus 1 blender with MSB + while (pos < limit64) + { + __m512i vidx = _mm512_loadu_epi8(indexes_pos + pos); + __m512i tmp1 = _mm512_permutex2var_epi8(table1, vidx, table2); + __m512i tmp2 = _mm512_permutex2var_epi8(table3, vidx, table4); + __mmask64 msb = _mm512_movepi8_mask(vidx); + __m512i out = _mm512_mask_blend_epi8(msb, tmp1, tmp2); + _mm512_storeu_epi8(res_pos + pos, out); + pos += 64; + } + if (limit > limit64) + { + __mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit); + __m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos); + __m512i tmp1 = _mm512_permutex2var_epi8(table1, vidx, table2); + __m512i tmp2 = _mm512_permutex2var_epi8(table3, vidx, table4); + __mmask64 msb = _mm512_movepi8_mask(vidx); + __m512i out = _mm512_mask_blend_epi8(msb, tmp1, tmp2); + _mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out); + } + } +} +); + template template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const @@ -399,8 +524,18 @@ ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_ auto res = this->create(limit); typename Self::Container & res_data = res->getData(); - for (size_t i = 0; i < limit; ++i) - res_data[i] = data[indexes[i]]; +#if USE_MULTITARGET_CODE + if constexpr (sizeof(T) == 1 && sizeof(Type) == 1) + { + /// VBMI optimization only applicable for (U)Int8 types + if (isArchSupported(TargetArch::AVX512VBMI)) + { + TargetSpecific::AVX512VBMI::vectorIndexImpl(data, indexes, limit, res_data); + return res; + } + } +#endif + TargetSpecific::Default::vectorIndexImpl(data, indexes, limit, res_data); return res; } diff --git a/src/Columns/tests/gtest_column_vector.cpp b/src/Columns/tests/gtest_column_vector.cpp new file mode 100644 index 00000000000..c5f0b3bd50a --- /dev/null +++ b/src/Columns/tests/gtest_column_vector.cpp @@ -0,0 +1,159 @@ +#include +#include +#include +#include +#include +#include + +using namespace DB; + +static pcg64 rng(randomSeed()); +static constexpr int error_code = 12345; +static constexpr size_t TEST_RUNS = 500; +static constexpr size_t MAX_ROWS = 10000; +static const std::vector filter_ratios = {1, 2, 5, 11, 32, 64, 100, 1000}; +static const size_t K = filter_ratios.size(); + +template +static MutableColumnPtr createColumn(size_t n) +{ + auto column = ColumnVector::create(); + auto & values = column->getData(); + + for (size_t i = 0; i < n; ++i) + { + values.push_back(i); + } + + return column; +} + +bool checkFilter(const PaddedPODArray &flit, const IColumn & src, const IColumn & dst) +{ + size_t n = flit.size(); + size_t dst_size = dst.size(); + size_t j = 0; /// index of dest + for (size_t i = 0; i < n; ++i) + { + if (flit[i] != 0) + { + if ((dst_size <= j) || (src.compareAt(i, j, dst, 0) != 0)) + return false; + j++; + } + } + return dst_size == j; /// filtered size check +} + +template +static void testFilter() +{ + auto test_case = [&](size_t rows, size_t filter_ratio) + { + auto vector_column = createColumn(rows); + PaddedPODArray flit(rows); + for (size_t i = 0; i < rows; ++i) + flit[i] = rng() % filter_ratio == 0; + auto res_column = vector_column->filter(flit, -1); + + if (!checkFilter(flit, *vector_column, *res_column)) + throw Exception(error_code, "VectorColumn filter failure, type: {}", typeid(T).name()); + }; + + try + { + for (size_t i = 0; i < TEST_RUNS; ++i) + { + size_t rows = rng() % MAX_ROWS + 1; + size_t filter_ratio = filter_ratios[rng() % K]; + + test_case(rows, filter_ratio); + } + } + catch (const Exception & e) + { + FAIL() << e.displayText(); + } +} + +TEST(ColumnVector, Filter) +{ + testFilter(); + testFilter(); + testFilter(); + testFilter(); + testFilter(); + testFilter(); + testFilter(); + testFilter(); + testFilter(); +} + +template +static MutableColumnPtr createIndexColumn(size_t limit, size_t rows) +{ + auto column = ColumnVector::create(); + auto & values = column->getData(); + auto max = std::numeric_limits::max(); + limit = limit > max ? max : limit; + + for (size_t i = 0; i < rows; ++i) + { + T val = rng() % limit; + values.push_back(val); + } + + return column; +} + +template +static void testIndex() +{ + static const std::vector column_sizes = {64, 128, 196, 256, 512}; + + auto test_case = [&](size_t rows, size_t index_rows, size_t limit) + { + auto vector_column = createColumn(rows); + auto index_column = createIndexColumn(rows, index_rows); + auto res_column = vector_column->index(*index_column, limit); + if (limit == 0) + limit = index_column->size(); + + /// check results + if (limit != res_column->size()) + throw Exception(error_code, "ColumnVector index size not match to limit: {} {}", typeid(T).name(), typeid(IndexType).name()); + for (size_t i = 0; i < limit; ++i) + { + /// vector_column data is the same as index, so indexed column's value will equals to index_column. + if (res_column->get64(i) != index_column->get64(i)) + throw Exception(error_code, "ColumnVector index fail: {} {}", typeid(T).name(), typeid(IndexType).name()); + } + }; + + try + { + test_case(0, 0, 0); /// test for zero length index + for (size_t i = 0; i < TEST_RUNS; ++i) + { + /// make sure rows distribute in (column_sizes[r-1], colulmn_sizes[r]] + size_t row_idx = rng() % column_sizes.size(); + size_t row_base = row_idx > 0 ? column_sizes[row_idx - 1] : 0; + size_t rows = row_base + (rng() % (column_sizes[row_idx] - row_base) + 1); + size_t index_rows = rng() % MAX_ROWS + 1; + + test_case(rows, index_rows, 0); + test_case(rows, index_rows, static_cast(0.5 * index_rows)); + } + } + catch (const Exception & e) + { + FAIL() << e.displayText(); + } +} + +TEST(ColumnVector, Index) +{ + testIndex(); + testIndex(); + testIndex(); +} diff --git a/src/Common/CpuId.h b/src/Common/CpuId.h index 167fa22faf6..1e54ccf62b3 100644 --- a/src/Common/CpuId.h +++ b/src/Common/CpuId.h @@ -82,6 +82,7 @@ inline bool cpuid(UInt32 op, UInt32 * res) noexcept /// NOLINT OP(AVX512BW) \ OP(AVX512VL) \ OP(AVX512VBMI) \ + OP(AVX512VBMI2) \ OP(PREFETCHWT1) \ OP(SHA) \ OP(ADX) \ @@ -302,6 +303,11 @@ bool haveAVX512VBMI() noexcept return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ecx >> 1) & 1u); } +bool haveAVX512VBMI2() noexcept +{ + return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ecx >> 6) & 1u); +} + bool haveRDRAND() noexcept { return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x1).registers.ecx >> 30) & 1u); diff --git a/src/Common/TargetSpecific.cpp b/src/Common/TargetSpecific.cpp index 9a445ea0fc1..1ab499027bf 100644 --- a/src/Common/TargetSpecific.cpp +++ b/src/Common/TargetSpecific.cpp @@ -21,6 +21,8 @@ UInt32 getSupportedArchs() result |= static_cast(TargetArch::AVX512BW); if (Cpu::CpuFlagsCache::have_AVX512VBMI) result |= static_cast(TargetArch::AVX512VBMI); + if (Cpu::CpuFlagsCache::have_AVX512VBMI2) + result |= static_cast(TargetArch::AVX512VBMI2); return result; } @@ -39,8 +41,9 @@ String toString(TargetArch arch) case TargetArch::AVX: return "avx"; case TargetArch::AVX2: return "avx2"; case TargetArch::AVX512F: return "avx512f"; - case TargetArch::AVX512BW: return "avx512bw"; - case TargetArch::AVX512VBMI: return "avx512vbmi"; + case TargetArch::AVX512BW: return "avx512bw"; + case TargetArch::AVX512VBMI: return "avx512vbmi"; + case TargetArch::AVX512VBMI2: return "avx512vbmi"; } UNREACHABLE(); diff --git a/src/Common/TargetSpecific.h b/src/Common/TargetSpecific.h index f078c0e3ffc..250642f6ee4 100644 --- a/src/Common/TargetSpecific.h +++ b/src/Common/TargetSpecific.h @@ -31,7 +31,7 @@ * int funcImpl() { * return 2; * } - * ) // DECLARE_DEFAULT_CODE + * ) // DECLARE_AVX2_SPECIFIC_CODE * * int func() { * #if USE_MULTITARGET_CODE @@ -80,8 +80,9 @@ enum class TargetArch : UInt32 AVX = (1 << 1), AVX2 = (1 << 2), AVX512F = (1 << 3), - AVX512BW = (1 << 4), - AVX512VBMI = (1 << 5), + AVX512BW = (1 << 4), + AVX512VBMI = (1 << 5), + AVX512VBMI2 = (1 << 6), }; /// Runtime detection. @@ -100,6 +101,7 @@ String toString(TargetArch arch); #if defined(__clang__) +#define AVX512VBMI2_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi,avx512vbmi2"))) #define AVX512VBMI_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi"))) #define AVX512BW_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw"))) #define AVX512_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f"))) @@ -108,6 +110,8 @@ String toString(TargetArch arch); #define SSE42_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt"))) #define DEFAULT_FUNCTION_SPECIFIC_ATTRIBUTE +# define BEGIN_AVX512VBMI2_SPECIFIC_CODE \ + _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi,avx512vbmi2\"))),apply_to=function)") # define BEGIN_AVX512VBMI_SPECIFIC_CODE \ _Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi\"))),apply_to=function)") # define BEGIN_AVX512BW_SPECIFIC_CODE \ @@ -129,6 +133,7 @@ String toString(TargetArch arch); # define DUMMY_FUNCTION_DEFINITION [[maybe_unused]] void _dummy_function_definition(); #else +#define AVX512VBMI2_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi,avx512vbmi2,tune=native"))) #define AVX512VBMI_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi,tune=native"))) #define AVX512BW_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,tune=native"))) #define AVX512_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,tune=native"))) @@ -137,6 +142,9 @@ String toString(TargetArch arch); #define SSE42_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt",tune=native))) #define DEFAULT_FUNCTION_SPECIFIC_ATTRIBUTE +# define BEGIN_AVX512VBMI2_SPECIFIC_CODE \ + _Pragma("GCC push_options") \ + _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi,avx512vbmi2,tune=native\")") # define BEGIN_AVX512VBMI_SPECIFIC_CODE \ _Pragma("GCC push_options") \ _Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi,tune=native\")") @@ -217,6 +225,16 @@ namespace TargetSpecific::AVX512VBMI { \ } \ END_TARGET_SPECIFIC_CODE +#define DECLARE_AVX512VBMI2_SPECIFIC_CODE(...) \ +BEGIN_AVX512VBMI2_SPECIFIC_CODE \ +namespace TargetSpecific::AVX512VBMI2 { \ + DUMMY_FUNCTION_DEFINITION \ + using namespace DB::TargetSpecific::AVX512VBMI2; \ + __VA_ARGS__ \ +} \ +END_TARGET_SPECIFIC_CODE + + #else #define USE_MULTITARGET_CODE 0 @@ -229,6 +247,7 @@ END_TARGET_SPECIFIC_CODE #define DECLARE_AVX512F_SPECIFIC_CODE(...) #define DECLARE_AVX512BW_SPECIFIC_CODE(...) #define DECLARE_AVX512VBMI_SPECIFIC_CODE(...) +#define DECLARE_AVX512VBMI2_SPECIFIC_CODE(...) #endif @@ -245,8 +264,9 @@ DECLARE_SSE42_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX2_SPECIFIC_CODE (__VA_ARGS__) \ DECLARE_AVX512F_SPECIFIC_CODE(__VA_ARGS__) \ -DECLARE_AVX512BW_SPECIFIC_CODE(__VA_ARGS__) \ -DECLARE_AVX512VBMI_SPECIFIC_CODE(__VA_ARGS__) +DECLARE_AVX512BW_SPECIFIC_CODE (__VA_ARGS__) \ +DECLARE_AVX512VBMI_SPECIFIC_CODE (__VA_ARGS__) \ +DECLARE_AVX512VBMI2_SPECIFIC_CODE (__VA_ARGS__) DECLARE_DEFAULT_CODE( constexpr auto BuildArch = TargetArch::Default; /// NOLINT @@ -276,6 +296,9 @@ DECLARE_AVX512VBMI_SPECIFIC_CODE( constexpr auto BuildArch = TargetArch::AVX512VBMI; /// NOLINT ) // DECLARE_AVX512VBMI_SPECIFIC_CODE +DECLARE_AVX512VBMI2_SPECIFIC_CODE( + constexpr auto BuildArch = TargetArch::AVX512VBMI2; /// NOLINT +) // DECLARE_AVX512VBMI2_SPECIFIC_CODE /** Runtime Dispatch helpers for class members. * From eed21ad4caa803b6ead095399d1e2fef83da3b5f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 25 Oct 2022 01:30:32 +0200 Subject: [PATCH 179/188] Revert "Low cardinality cases moved to the function for its corresponding type" --- src/Functions/FunctionsJSON.cpp | 123 +++--------------- .../performance/low_cardinality_from_json.xml | 73 ----------- .../02452_check_low_cardinality.reference | 8 -- .../02452_check_low_cardinality.sql | 73 ----------- ...ct_fixed_string_from_nested_json.reference | 1 - ..._extract_fixed_string_from_nested_json.sql | 6 - 6 files changed, 21 insertions(+), 263 deletions(-) delete mode 100644 tests/performance/low_cardinality_from_json.xml delete mode 100644 tests/queries/0_stateless/02452_check_low_cardinality.reference delete mode 100644 tests/queries/0_stateless/02452_check_low_cardinality.sql delete mode 100644 tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.reference delete mode 100644 tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.sql diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 2f0a0889847..493fc36ca3c 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -23,7 +23,6 @@ #include #include #include -#include #include #include #include @@ -696,16 +695,8 @@ public: else return false; - if (dest.getDataType() == TypeIndex::LowCardinality) - { - ColumnLowCardinality & col_low = assert_cast(dest); - col_low.insertData(reinterpret_cast(&value), sizeof(value)); - } - else - { - auto & col_vec = assert_cast &>(dest); - col_vec.insertValue(value); - } + auto & col_vec = assert_cast &>(dest); + col_vec.insertValue(value); return true; } }; @@ -782,17 +773,8 @@ public: return JSONExtractRawImpl::insertResultToColumn(dest, element, {}); auto str = element.getString(); - - if (dest.getDataType() == TypeIndex::LowCardinality) - { - ColumnLowCardinality & col_low = assert_cast(dest); - col_low.insertData(str.data(), str.size()); - } - else - { - ColumnString & col_str = assert_cast(dest); - col_str.insertData(str.data(), str.size()); - } + ColumnString & col_str = assert_cast(dest); + col_str.insertData(str.data(), str.size()); return true; } }; @@ -821,33 +803,25 @@ struct JSONExtractTree } }; - class LowCardinalityFixedStringNode : public Node + class LowCardinalityNode : public Node { public: - explicit LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) {} + LowCardinalityNode(DataTypePtr dictionary_type_, std::unique_ptr impl_) + : dictionary_type(dictionary_type_), impl(std::move(impl_)) {} bool insertResultToColumn(IColumn & dest, const Element & element) override { - // If element is an object we delegate the insertion to JSONExtractRawImpl - if (element.isObject()) - return JSONExtractRawImpl::insertResultToLowCardinalityFixedStringColumn(dest, element, fixed_length); - else if (!element.isString()) - return false; - - auto str = element.getString(); - if (str.size() > fixed_length) - return false; - - // For the non low cardinality case of FixedString, the padding is done in the FixedString Column implementation. - // In order to avoid having to pass the data to a FixedString Column and read it back (which would slow down the execution) - // the data is padded here and written directly to the Low Cardinality Column - auto padded_str = str.data() + std::string(fixed_length - std::min(fixed_length, str.length()), '\0'); - - assert_cast(dest).insertData(padded_str.data(), padded_str.size()); - return true; + auto from_col = dictionary_type->createColumn(); + if (impl->insertResultToColumn(*from_col, element)) + { + std::string_view value = from_col->getDataAt(0).toView(); + assert_cast(dest).insertData(value.data(), value.size()); + return true; + } + return false; } - private: - const size_t fixed_length; + DataTypePtr dictionary_type; + std::unique_ptr impl; }; class UUIDNode : public Node @@ -859,15 +833,7 @@ struct JSONExtractTree return false; auto uuid = parseFromString(element.getString()); - if (dest.getDataType() == TypeIndex::LowCardinality) - { - ColumnLowCardinality & col_low = assert_cast(dest); - col_low.insertData(reinterpret_cast(&uuid), sizeof(uuid)); - } - else - { - assert_cast(dest).insert(uuid); - } + assert_cast(dest).insert(uuid); return true; } }; @@ -887,7 +853,6 @@ struct JSONExtractTree assert_cast &>(dest).insert(result); return true; } - private: DataTypePtr data_type; }; @@ -906,18 +871,13 @@ struct JSONExtractTree public: bool insertResultToColumn(IColumn & dest, const Element & element) override { - if (element.isNull()) - return false; - if (!element.isString()) - return JSONExtractRawImpl::insertResultToFixedStringColumn(dest, element, {}); - - auto str = element.getString(); + return false; auto & col_str = assert_cast(dest); + auto str = element.getString(); if (str.size() > col_str.getN()) return false; col_str.insertData(str.data(), str.size()); - return true; } }; @@ -1139,19 +1099,9 @@ struct JSONExtractTree case TypeIndex::UUID: return std::make_unique(); case TypeIndex::LowCardinality: { - // The low cardinality case is treated in two different ways: - // For FixedString type, an especial class is implemented for inserting the data in the destination column, - // as the string length must be passed in order to check and pad the incoming data. - // For the rest of low cardinality types, the insertion is done in their corresponding class, adapting the data - // as needed for the insertData function of the ColumnLowCardinality. auto dictionary_type = typeid_cast(type.get())->getDictionaryType(); - if ((*dictionary_type).getTypeId() == TypeIndex::FixedString) - { - auto fixed_length = typeid_cast(dictionary_type.get())->getN(); - return std::make_unique(fixed_length); - } auto impl = build(function_name, dictionary_type); - return impl; + return std::make_unique(dictionary_type, std::move(impl)); } case TypeIndex::Decimal256: return std::make_unique>(type); case TypeIndex::Decimal128: return std::make_unique>(type); @@ -1313,37 +1263,6 @@ public: return true; } - // We use insertResultToFixedStringColumn in case we are inserting raw data in a FixedString column - static bool insertResultToFixedStringColumn(IColumn & dest, const Element & element, std::string_view) - { - ColumnFixedString & col_str = assert_cast(dest); - auto & chars = col_str.getChars(); - WriteBufferFromVector buf(chars, AppendModeTag()); - traverse(element, buf); - buf.finalize(); - col_str.insertDefault(); - return true; - } - - // We use insertResultToLowCardinalityFixedStringColumn in case we are inserting raw data in a Low Cardinality FixedString column - static bool insertResultToLowCardinalityFixedStringColumn(IColumn & dest, const Element & element, size_t fixed_length) - { - if (element.getObject().size() > fixed_length) - return false; - - ColumnFixedString::Chars chars; - WriteBufferFromVector buf(chars, AppendModeTag()); - traverse(element, buf); - buf.finalize(); - chars.push_back(0); - std::string str = reinterpret_cast(chars.data()); - - auto padded_str = str + std::string(fixed_length - std::min(fixed_length, str.length()), '\0'); - assert_cast(dest).insertData(padded_str.data(), padded_str.size()); - - return true; - } - private: static void traverse(const Element & element, WriteBuffer & buf) { diff --git a/tests/performance/low_cardinality_from_json.xml b/tests/performance/low_cardinality_from_json.xml deleted file mode 100644 index ac6542ac503..00000000000 --- a/tests/performance/low_cardinality_from_json.xml +++ /dev/null @@ -1,73 +0,0 @@ - - - - - string_json - - '{"a": "hi", "b": "hello", "c": "hola", "d": "see you, bye, bye"}' - - - - int_json - - '{"a": 11, "b": 2222, "c": 33333333, "d": 4444444444444444}' - - - - uuid_json - - '{"a": "2d49dc6e-ddce-4cd0-afb8-790956df54c4", "b": "2d49dc6e-ddce-4cd0-afb8-790956df54c3", "c": "2d49dc6e-ddce-4cd0-afb8-790956df54c1", "d": "2d49dc6e-ddce-4cd0-afb8-790956df54c1"}' - - - - low_cardinality_tuple_string - - 'Tuple(a LowCardinality(String), b LowCardinality(String), c LowCardinality(String), d LowCardinality(String) )' - - - - low_cardinality_tuple_fixed_string - - 'Tuple(a LowCardinality(FixedString(20)), b LowCardinality(FixedString(20)), c LowCardinality(FixedString(20)), d LowCardinality(FixedString(20)) )' - - - - low_cardinality_tuple_int8 - - 'Tuple(a LowCardinality(Int8), b LowCardinality(Int8), c LowCardinality(Int8), d LowCardinality(Int8) )' - - - - low_cardinality_tuple_int16 - - 'Tuple(a LowCardinality(Int16), b LowCardinality(Int16), c LowCardinality(Int16), d LowCardinality(Int16) )' - - - - low_cardinality_tuple_int32 - - 'Tuple(a LowCardinality(Int32), b LowCardinality(Int32), c LowCardinality(Int32), d LowCardinality(Int32) )' - - - - low_cardinality_tuple_int64 - - 'Tuple(a LowCardinality(Int64), b LowCardinality(Int64), c LowCardinality(Int64), d LowCardinality(Int64) )' - - - - low_cardinality_tuple_uuid - - 'Tuple(a LowCardinality(UUID), b LowCardinality(UUID), c LowCardinality(UUID), d LowCardinality(UUID) )' - - - - - SELECT 'fixed_string_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({string_json}), {low_cardinality_tuple_fixed_string})) FORMAT Null - SELECT 'string_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({string_json}), {low_cardinality_tuple_string})) FORMAT Null - SELECT 'int8_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int8})) FORMAT Null - SELECT 'int16_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int16})) FORMAT Null - SELECT 'int32_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int32})) FORMAT Null - SELECT 'int64_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int64})) FORMAT Null - SELECT 'uuid_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({uuid_json}), {low_cardinality_tuple_uuid})) FORMAT Null - \ No newline at end of file diff --git a/tests/queries/0_stateless/02452_check_low_cardinality.reference b/tests/queries/0_stateless/02452_check_low_cardinality.reference deleted file mode 100644 index dc5553d7886..00000000000 --- a/tests/queries/0_stateless/02452_check_low_cardinality.reference +++ /dev/null @@ -1,8 +0,0 @@ -('hi','hello','hola','see you, bye, bye') -('hi\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','hello\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','hola\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','see you, bye, bye\0\0\0') -(11,0,0,0) -(11,2222,0,0) -(11,2222,33333333,0) -(11,2222,33333333,4444444444444444) -('2d49dc6e-ddce-4cd0-afb8-790956df54c4','2d49dc6e-ddce-4cd0-afb8-790956df54c3','2d49dc6e-ddce-4cd0-afb8-790956df54c1','2d49dc6e-ddce-4cd0-afb8-790956df54c1') -('{"b":{"c":1,"d":"str"}}\0\0\0\0\0\0\0','\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0') diff --git a/tests/queries/0_stateless/02452_check_low_cardinality.sql b/tests/queries/0_stateless/02452_check_low_cardinality.sql deleted file mode 100644 index d62f418139e..00000000000 --- a/tests/queries/0_stateless/02452_check_low_cardinality.sql +++ /dev/null @@ -1,73 +0,0 @@ --- Tags: no-fasttest -DROP TABLE IF EXISTS test_low_cardinality_string; -DROP TABLE IF EXISTS test_low_cardinality_string_nested_json; -DROP TABLE IF EXISTS test_low_cardinality_uuid; -DROP TABLE IF EXISTS test_low_cardinality_int; -CREATE TABLE test_low_cardinality_string (data String) ENGINE MergeTree ORDER BY data; -CREATE TABLE test_low_cardinality_string_nested_json (data String) ENGINE MergeTree ORDER BY data; -CREATE TABLE test_low_cardinality_uuid (data String) ENGINE MergeTree ORDER BY data; -CREATE TABLE test_low_cardinality_int (data String) ENGINE MergeTree ORDER BY data; -INSERT INTO test_low_cardinality_string (data) VALUES ('{"a": "hi", "b": "hello", "c": "hola", "d": "see you, bye, bye"}'); -INSERT INTO test_low_cardinality_string_nested_json (data) VALUES ('{"a" : {"b" : {"c" : 1, "d" : "str"}}}'); -INSERT INTO test_low_cardinality_int (data) VALUES ('{"a": 11, "b": 2222, "c": 33333333, "d": 4444444444444444}'); -INSERT INTO test_low_cardinality_uuid (data) VALUES ('{"a": "2d49dc6e-ddce-4cd0-afb8-790956df54c4", "b": "2d49dc6e-ddce-4cd0-afb8-790956df54c3", "c": "2d49dc6e-ddce-4cd0-afb8-790956df54c1", "d": "2d49dc6e-ddce-4cd0-afb8-790956df54c1"}'); -SELECT JSONExtract(data, 'Tuple( - a LowCardinality(String), - b LowCardinality(String), - c LowCardinality(String), - d LowCardinality(String) - - )') AS json FROM test_low_cardinality_string; -SELECT JSONExtract(data, 'Tuple( - a LowCardinality(FixedString(20)), - b LowCardinality(FixedString(20)), - c LowCardinality(FixedString(20)), - d LowCardinality(FixedString(20)) - - )') AS json FROM test_low_cardinality_string; -SELECT JSONExtract(data, 'Tuple( - a LowCardinality(Int8), - b LowCardinality(Int8), - c LowCardinality(Int8), - d LowCardinality(Int8) - - )') AS json FROM test_low_cardinality_int; -SELECT JSONExtract(data, 'Tuple( - a LowCardinality(Int16), - b LowCardinality(Int16), - c LowCardinality(Int16), - d LowCardinality(Int16) - - )') AS json FROM test_low_cardinality_int; -SELECT JSONExtract(data, 'Tuple( - a LowCardinality(Int32), - b LowCardinality(Int32), - c LowCardinality(Int32), - d LowCardinality(Int32) - - )') AS json FROM test_low_cardinality_int; -SELECT JSONExtract(data, 'Tuple( - a LowCardinality(Int64), - b LowCardinality(Int64), - c LowCardinality(Int64), - d LowCardinality(Int64) - - )') AS json FROM test_low_cardinality_int; -SELECT JSONExtract(data, 'Tuple( - a LowCardinality(UUID), - b LowCardinality(UUID), - c LowCardinality(UUID), - d LowCardinality(UUID) - - )') AS json FROM test_low_cardinality_uuid; -SELECT JSONExtract('{"a" : {"b" : {"c" : 1, "d" : "str"}}}', 'Tuple( - a LowCardinality(FixedString(30)), - b LowCardinality(FixedString(20)), - c LowCardinality(FixedString(20)), - d LowCardinality(FixedString(20)) - - )') AS json FROM test_low_cardinality_string; -DROP TABLE test_low_cardinality_string_nested_json; -DROP TABLE test_low_cardinality_string; -DROP TABLE test_low_cardinality_uuid; -DROP TABLE test_low_cardinality_int; diff --git a/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.reference b/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.reference deleted file mode 100644 index 3a528a24821..00000000000 --- a/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.reference +++ /dev/null @@ -1 +0,0 @@ -('{"b":{"c":1,"d":"str"}}\0') diff --git a/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.sql b/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.sql deleted file mode 100644 index c3eee60f161..00000000000 --- a/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.sql +++ /dev/null @@ -1,6 +0,0 @@ --- Tags: no-fasttest -DROP TABLE IF EXISTS test_fixed_string_nested_json; -CREATE TABLE test_fixed_string_nested_json (data String) ENGINE MergeTree ORDER BY data; -INSERT INTO test_fixed_string_nested_json (data) VALUES ('{"a" : {"b" : {"c" : 1, "d" : "str"}}}'); -SELECT JSONExtract(data, 'Tuple(a FixedString(24))') AS json FROM test_fixed_string_nested_json; -DROP TABLE test_fixed_string_nested_json; From e30d425869ae332d3b1c2ddf3f85866355d4e27c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 25 Oct 2022 01:55:02 +0000 Subject: [PATCH 180/188] fix build --- src/Columns/tests/gtest_column_vector.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Columns/tests/gtest_column_vector.cpp b/src/Columns/tests/gtest_column_vector.cpp index c5f0b3bd50a..5017d687791 100644 --- a/src/Columns/tests/gtest_column_vector.cpp +++ b/src/Columns/tests/gtest_column_vector.cpp @@ -21,9 +21,7 @@ static MutableColumnPtr createColumn(size_t n) auto & values = column->getData(); for (size_t i = 0; i < n; ++i) - { - values.push_back(i); - } + values.push_back(static_cast(i)); return column; } From 646f2f8f9c5ad4e1b018c5efec9bde6819049984 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Oct 2022 05:20:16 +0200 Subject: [PATCH 181/188] Add .rgignore for test data --- tests/.rgignore | 1 + 1 file changed, 1 insertion(+) create mode 100644 tests/.rgignore diff --git a/tests/.rgignore b/tests/.rgignore new file mode 100644 index 00000000000..26cb6f9025d --- /dev/null +++ b/tests/.rgignore @@ -0,0 +1 @@ +data_json From 62a33b12d50e7dd263fa9c0b5b573ac5b666ec21 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 25 Oct 2022 11:24:42 +0200 Subject: [PATCH 182/188] Small refactor --- src/Parsers/ExpressionListParsers.cpp | 30 +++++++++++++-------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index a13282bc3d5..4f5438d5b5c 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -786,10 +786,10 @@ public: /// 2. If we parse an ending token, we should merge everything as in (1) and /// also set 'finished' flag. template -class BaseLayer : public Layer +class LayerWithSeparator : public Layer { public: - explicit BaseLayer(bool allow_alias_ = true, bool allow_alias_without_as_keyword_ = false) : + explicit LayerWithSeparator(bool allow_alias_ = true, bool allow_alias_without_as_keyword_ = false) : Layer(allow_alias_, allow_alias_without_as_keyword_) {} bool parse(IParser::Pos & pos, Expected & expected, Action & action) override @@ -1066,7 +1066,7 @@ private: }; /// Layer for array square brackets operator -class ArrayLayer : public BaseLayer +class ArrayLayer : public LayerWithSeparator { public: bool getResult(ASTPtr & node) override @@ -1077,19 +1077,19 @@ public: bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { - return BaseLayer::parse(pos, expected, action); + return LayerWithSeparator::parse(pos, expected, action); } }; /// Layer for arrayElement square brackets operator /// This layer does not create a function, it is only needed to parse closing token /// and return only one element. -class ArrayElementLayer : public BaseLayer +class ArrayElementLayer : public LayerWithSeparator { public: bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { - return BaseLayer::parse(pos, expected, action); + return LayerWithSeparator::parse(pos, expected, action); } }; @@ -1193,10 +1193,10 @@ public: } }; -class ExtractLayer : public BaseLayer +class ExtractLayer : public LayerWithSeparator { public: - ExtractLayer() : BaseLayer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} + ExtractLayer() : LayerWithSeparator(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} bool getResult(ASTPtr & node) override { @@ -1242,7 +1242,7 @@ public: if (state == 1) { - return BaseLayer::parse(pos, expected, action); + return LayerWithSeparator::parse(pos, expected, action); } if (state == 2) @@ -1584,11 +1584,11 @@ private: String function_name; }; -class DateAddLayer : public BaseLayer +class DateAddLayer : public LayerWithSeparator { public: explicit DateAddLayer(const char * function_name_) - : BaseLayer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true), function_name(function_name_) {} + : LayerWithSeparator(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true), function_name(function_name_) {} bool getResult(ASTPtr & node) override { @@ -1630,7 +1630,7 @@ public: if (state == 1) { - return BaseLayer::parse(pos, expected, action); + return LayerWithSeparator::parse(pos, expected, action); } return true; @@ -1642,10 +1642,10 @@ private: bool parsed_interval_kind = false; }; -class DateDiffLayer : public BaseLayer +class DateDiffLayer : public LayerWithSeparator { public: - DateDiffLayer() : BaseLayer(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} + DateDiffLayer() : LayerWithSeparator(/*allow_alias*/ true, /*allow_alias_without_as_keyword*/ true) {} bool getResult(ASTPtr & node) override { @@ -1685,7 +1685,7 @@ public: if (state == 1) { - return BaseLayer::parse(pos, expected, action); + return LayerWithSeparator::parse(pos, expected, action); } return true; From 0016bc2b7398d93c1098dd005016bf9d8eae2863 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 25 Oct 2022 13:26:29 +0200 Subject: [PATCH 183/188] Improve the time to recover keeper connections (#42541) Co-authored-by: Nikita Mikhaylov Co-authored-by: robot-clickhouse Co-authored-by: Alexey Milovidov --- src/Common/ZooKeeper/ZooKeeper.cpp | 31 +++++-- src/Common/ZooKeeper/ZooKeeper.h | 10 ++- src/Interpreters/Context.cpp | 5 ++ src/Interpreters/DDLWorker.cpp | 8 +- .../ReplicatedMergeTreeRestartingThread.cpp | 63 ++++++------- .../ReplicatedMergeTreeRestartingThread.h | 1 + .../test_read_only_table/__init__.py | 0 .../integration/test_read_only_table/test.py | 89 +++++++++++++++++++ 8 files changed, 157 insertions(+), 50 deletions(-) create mode 100644 tests/integration/test_read_only_table/__init__.py create mode 100644 tests/integration/test_read_only_table/test.py diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 406d8b27c39..ab5d918e1f0 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -777,19 +777,34 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & return false; } -void ZooKeeper::waitForEphemeralToDisappearIfAny(const std::string & path) +void ZooKeeper::handleEphemeralNodeExistence(const std::string & path, const std::string & fast_delete_if_equal_value) { zkutil::EventPtr eph_node_disappeared = std::make_shared(); String content; - if (!tryGet(path, content, nullptr, eph_node_disappeared)) + Coordination::Stat stat; + if (!tryGet(path, content, &stat, eph_node_disappeared)) return; - int32_t timeout_ms = 3 * args.session_timeout_ms; - if (!eph_node_disappeared->tryWait(timeout_ms)) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, - "Ephemeral node {} still exists after {}s, probably it's owned by someone else. " - "Either session_timeout_ms in client's config is different from server's config or it's a bug. " - "Node data: '{}'", path, timeout_ms / 1000, content); + if (content == fast_delete_if_equal_value) + { + auto code = tryRemove(path, stat.version); + if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) + throw Coordination::Exception(code, path); + } + else + { + LOG_WARNING(log, "Ephemeral node ('{}') already exists but it isn't owned by us. Will wait until it disappears", path); + int32_t timeout_ms = 3 * args.session_timeout_ms; + if (!eph_node_disappeared->tryWait(timeout_ms)) + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, + "Ephemeral node {} still exists after {}s, probably it's owned by someone else. " + "Either session_timeout_ms in client's config is different from server's config or it's a bug. " + "Node data: '{}'", + path, + timeout_ms / 1000, + content); + } } ZooKeeperPtr ZooKeeper::startNewSession() const diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index f7d5bccc0e0..968d10ad9a5 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -393,9 +393,11 @@ public: /// The function returns true if waited and false if waiting was interrupted by condition. bool waitForDisappear(const std::string & path, const WaitCondition & condition = {}); - /// Wait for the ephemeral node created in previous session to disappear. - /// Throws LOGICAL_ERROR if node still exists after 2x session_timeout. - void waitForEphemeralToDisappearIfAny(const std::string & path); + /// Checks if a the ephemeral node exists. These nodes are removed automatically by ZK when the session ends + /// If the node exists and its value is equal to fast_delete_if_equal_value it will remove it + /// If the node exists and its value is different, it will wait for it to disappear. It will throw a LOGICAL_ERROR if the node doesn't + /// disappear automatically after 3x session_timeout. + void handleEphemeralNodeExistence(const std::string & path, const std::string & fast_delete_if_equal_value); /// Async interface (a small subset of operations is implemented). /// @@ -609,7 +611,7 @@ public: catch (...) { ProfileEvents::increment(ProfileEvents::CannotRemoveEphemeralNode); - DB::tryLogCurrentException(__PRETTY_FUNCTION__, "Cannot remove " + path + ": "); + DB::tryLogCurrentException(__PRETTY_FUNCTION__, "Cannot remove " + path); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 721d701c9a2..386b39a86f2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2061,7 +2061,12 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const if (!shared->zookeeper) shared->zookeeper = std::make_shared(config, "zookeeper", getZooKeeperLog()); else if (shared->zookeeper->expired()) + { + Stopwatch watch; + LOG_DEBUG(shared->log, "Trying to establish a new connection with ZooKeeper"); shared->zookeeper = shared->zookeeper->startNewSession(); + LOG_DEBUG(shared->log, "Establishing a new connection with ZooKeeper took {} ms", watch.elapsedMilliseconds()); + } return shared->zookeeper; } diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index f1fa0955c34..6bfa9ecd591 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -532,7 +533,8 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper) auto active_node = zkutil::EphemeralNodeHolder::existing(active_node_path, *zookeeper); /// Try fast path - auto create_active_res = zookeeper->tryCreate(active_node_path, {}, zkutil::CreateMode::Ephemeral); + const String canary_value = Field(ServerUUID::get()).dump(); + auto create_active_res = zookeeper->tryCreate(active_node_path, canary_value, zkutil::CreateMode::Ephemeral); if (create_active_res != Coordination::Error::ZOK) { if (create_active_res != Coordination::Error::ZNONODE && create_active_res != Coordination::Error::ZNODEEXISTS) @@ -563,10 +565,10 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper) { /// Connection has been lost and now we are retrying, /// but our previous ephemeral node still exists. - zookeeper->waitForEphemeralToDisappearIfAny(active_node_path); + zookeeper->handleEphemeralNodeExistence(active_node_path, canary_value); } - zookeeper->create(active_node_path, {}, zkutil::CreateMode::Ephemeral); + zookeeper->create(active_node_path, canary_value, zkutil::CreateMode::Ephemeral); } /// We must hold the lock until task execution status is committed to ZooKeeper, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index e2b23d75746..10ec4702b53 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -26,19 +27,12 @@ namespace DB namespace ErrorCodes { extern const int REPLICA_IS_ALREADY_ACTIVE; - extern const int REPLICA_STATUS_CHANGED; - -} - -namespace -{ - constexpr auto retry_period_ms = 1000; } /// Used to check whether it's us who set node `is_active`, or not. static String generateActiveNodeIdentifier() { - return "pid: " + toString(getpid()) + ", random: " + toString(randomSeed()); + return Field(ServerUUID::get()).dump(); } ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_) @@ -58,27 +52,34 @@ void ReplicatedMergeTreeRestartingThread::run() if (need_stop) return; - size_t reschedule_period_ms = check_period_ms; + /// In case of any exceptions we want to rerun the this task as fast as possible but we also don't want to keep retrying immediately + /// in a close loop (as fast as tasks can be processed), so we'll retry in between 100 and 10000 ms + const size_t backoff_ms = 100 * ((consecutive_check_failures + 1) * (consecutive_check_failures + 2)) / 2; + const size_t next_failure_retry_ms = std::min(size_t{10000}, backoff_ms); try { bool replica_is_active = runImpl(); - if (!replica_is_active) - reschedule_period_ms = retry_period_ms; - } - catch (const Exception & e) - { - /// We couldn't activate table let's set it into readonly mode - partialShutdown(); - tryLogCurrentException(log, __PRETTY_FUNCTION__); - - if (e.code() == ErrorCodes::REPLICA_STATUS_CHANGED) - reschedule_period_ms = 0; + if (replica_is_active) + { + consecutive_check_failures = 0; + task->scheduleAfter(check_period_ms); + } + else + { + consecutive_check_failures++; + task->scheduleAfter(next_failure_retry_ms); + } } catch (...) { + consecutive_check_failures++; + task->scheduleAfter(next_failure_retry_ms); + + /// We couldn't activate table let's set it into readonly mode if necessary + /// We do this after scheduling the task in case it throws partialShutdown(); - tryLogCurrentException(log, __PRETTY_FUNCTION__); + tryLogCurrentException(log, "Failed to restart the table. Will try again"); } if (first_time) @@ -92,14 +93,6 @@ void ReplicatedMergeTreeRestartingThread::run() storage.startup_event.set(); first_time = false; } - - if (need_stop) - return; - - if (reschedule_period_ms) - task->scheduleAfter(reschedule_period_ms); - else - task->schedule(); } bool ReplicatedMergeTreeRestartingThread::runImpl() @@ -132,8 +125,8 @@ bool ReplicatedMergeTreeRestartingThread::runImpl() } catch (const Coordination::Exception &) { - /// The exception when you try to zookeeper_init usually happens if DNS does not work. We will try to do it again. - tryLogCurrentException(log, __PRETTY_FUNCTION__); + /// The exception when you try to zookeeper_init usually happens if DNS does not work or the connection with ZK fails + tryLogCurrentException(log, "Failed to establish a new ZK connection. Will try again"); assert(storage.is_readonly); return false; } @@ -158,12 +151,15 @@ bool ReplicatedMergeTreeRestartingThread::runImpl() storage.cleanup_thread.start(); storage.part_check_thread.start(); + LOG_DEBUG(log, "Table started successfully"); + return true; } bool ReplicatedMergeTreeRestartingThread::tryStartup() { + LOG_DEBUG(log, "Trying to start replica up"); try { removeFailedQuorumParts(); @@ -177,9 +173,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() try { storage.queue.initialize(zookeeper); - storage.queue.load(zookeeper); - storage.queue.createLogEntriesToFetchBrokenParts(); /// pullLogsToQueue() after we mark replica 'is_active' (and after we repair if it was lost); @@ -302,7 +296,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() ReplicatedMergeTreeAddress address = storage.getReplicatedMergeTreeAddress(); String is_active_path = fs::path(storage.replica_path) / "is_active"; - zookeeper->waitForEphemeralToDisappearIfAny(is_active_path); + zookeeper->handleEphemeralNodeExistence(is_active_path, active_node_identifier); /// Simultaneously declare that this replica is active, and update the host. Coordination::Requests ops; @@ -348,7 +342,6 @@ void ReplicatedMergeTreeRestartingThread::partialShutdown(bool part_of_full_shut storage.replica_is_active_node = nullptr; LOG_TRACE(log, "Waiting for threads to finish"); - storage.merge_selecting_task->deactivate(); storage.queue_updating_task->deactivate(); storage.mutations_updating_task->deactivate(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 3d443a236ed..bb4b0c0fdd2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -41,6 +41,7 @@ private: BackgroundSchedulePool::TaskHolder task; Int64 check_period_ms; /// The frequency of checking expiration of session in ZK. + UInt32 consecutive_check_failures = 0; /// How many consecutive checks have failed bool first_time = true; /// Activate replica for the first time. void run(); diff --git a/tests/integration/test_read_only_table/__init__.py b/tests/integration/test_read_only_table/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_read_only_table/test.py b/tests/integration/test_read_only_table/test.py new file mode 100644 index 00000000000..28abbf6601e --- /dev/null +++ b/tests/integration/test_read_only_table/test.py @@ -0,0 +1,89 @@ +import time +import re +import logging + +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +NUM_TABLES = 10 + + +def fill_nodes(nodes): + for table_id in range(NUM_TABLES): + for node in nodes: + node.query( + f""" + CREATE TABLE test_table_{table_id}(a UInt64) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/replicated/{table_id}', '{node.name}') ORDER BY tuple(); + """ + ) + + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance("node1", with_zookeeper=True) +node2 = cluster.add_instance("node2", with_zookeeper=True) +node3 = cluster.add_instance("node3", with_zookeeper=True) +nodes = [node1, node2, node3] + + +def sync_replicas(table): + for node in nodes: + node.query(f"SYSTEM SYNC REPLICA {table}") + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + fill_nodes(nodes) + + yield cluster + + except Exception as ex: + print(ex) + + finally: + cluster.shutdown() + + +def test_restart_zookeeper(start_cluster): + + for table_id in range(NUM_TABLES): + node1.query( + f"INSERT INTO test_table_{table_id} VALUES (1), (2), (3), (4), (5);" + ) + + logging.info("Inserted test data and initialized all tables") + + def get_zookeeper_which_node_connected_to(node): + line = str( + node.exec_in_container( + [ + "bash", + "-c", + "lsof -a -i4 -i6 -itcp -w | grep 2181 | grep ESTABLISHED", + ], + privileged=True, + user="root", + ) + ).strip() + + pattern = re.compile(r"zoo[0-9]+", re.IGNORECASE) + result = pattern.findall(line) + assert ( + len(result) == 1 + ), "ClickHouse must be connected only to one Zookeeper at a time" + return result[0] + + node1_zk = get_zookeeper_which_node_connected_to(node1) + + # ClickHouse should +- immediately reconnect to another zookeeper node + cluster.stop_zookeeper_nodes([node1_zk]) + time.sleep(5) + + for table_id in range(NUM_TABLES): + node1.query( + f"INSERT INTO test_table_{table_id} VALUES (6), (7), (8), (9), (10);" + ) From 49f6692a2ec808f5518045f2046de00d80bf4094 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 25 Oct 2022 13:56:28 +0200 Subject: [PATCH 184/188] Adapt internal data structures to 512-bit era (#42564) * impl * update tests * fix tests --- src/Columns/ColumnVectorHelper.h | 8 ++- src/Common/Arena.h | 3 +- src/Common/PODArray.cpp | 17 +++-- src/Common/PODArray.h | 19 +++--- src/Common/PODArray_fwd.h | 3 +- src/Common/tests/gtest_pod_array.cpp | 2 +- src/Core/Defines.h | 7 +- src/Functions/FunctionsRandom.cpp | 2 +- src/IO/BufferWithOwnMemory.h | 3 +- src/IO/MMapReadBufferFromFileDescriptor.cpp | 2 +- src/IO/MMapReadBufferFromFileWithCache.cpp | 2 +- src/IO/PeekableReadBuffer.h | 2 +- src/IO/tests/gtest_memory_resize.cpp | 65 +++++++++---------- tests/integration/test_grpc_protocol/test.py | 2 +- ...m_columns_and_system_tables_long.reference | 2 +- .../02136_scalar_progress.reference | 4 +- .../02373_progress_contain_result.reference | 2 +- .../02458_default_setting.reference | 4 +- 18 files changed, 76 insertions(+), 73 deletions(-) diff --git a/src/Columns/ColumnVectorHelper.h b/src/Columns/ColumnVectorHelper.h index 36cbfbf640e..b8ea6ca427f 100644 --- a/src/Columns/ColumnVectorHelper.h +++ b/src/Columns/ColumnVectorHelper.h @@ -28,13 +28,17 @@ public: template const char * getRawDataBegin() const { - return reinterpret_cast, 15, 16> *>(reinterpret_cast(this) + sizeof(*this))->raw_data(); + return reinterpret_cast, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD> *>( + reinterpret_cast(this) + sizeof(*this)) + ->raw_data(); } template void insertRawData(const char * ptr) { - return reinterpret_cast, 15, 16> *>(reinterpret_cast(this) + sizeof(*this))->push_back_raw(ptr); + return reinterpret_cast, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD> *>( + reinterpret_cast(this) + sizeof(*this)) + ->push_back_raw(ptr); } }; diff --git a/src/Common/Arena.h b/src/Common/Arena.h index b706f3b3413..17d53acd8f7 100644 --- a/src/Common/Arena.h +++ b/src/Common/Arena.h @@ -34,8 +34,7 @@ namespace DB class Arena : private boost::noncopyable { private: - /// Padding allows to use 'memcpySmallAllowReadWriteOverflow15' instead of 'memcpy'. - static constexpr size_t pad_right = 15; + static constexpr size_t pad_right = PADDING_FOR_SIMD - 1; /// Contiguous MemoryChunk of memory and pointer to free space inside it. Member of single-linked list. struct alignas(16) MemoryChunk : private Allocator /// empty base optimization diff --git a/src/Common/PODArray.cpp b/src/Common/PODArray.cpp index c1edc5bafad..07c3cf1af1a 100644 --- a/src/Common/PODArray.cpp +++ b/src/Common/PODArray.cpp @@ -6,14 +6,13 @@ namespace DB /// Used for left padding of PODArray when empty const char empty_pod_array[empty_pod_array_size]{}; -template class PODArray, 15, 16>; -template class PODArray, 15, 16>; -template class PODArray, 15, 16>; -template class PODArray, 15, 16>; - -template class PODArray, 15, 16>; -template class PODArray, 15, 16>; -template class PODArray, 15, 16>; -template class PODArray, 15, 16>; +template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; } diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index 0baefad39e2..ea3115677fc 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -502,7 +502,7 @@ public: template void insertSmallAllowReadWriteOverflow15(It1 from_begin, It2 from_end, TAllocatorParams &&... allocator_params) { - static_assert(pad_right_ >= 15); + static_assert(pad_right_ >= PADDING_FOR_SIMD - 1); static_assert(sizeof(T) == sizeof(*from_begin)); insertPrepare(from_begin, from_end, std::forward(allocator_params)...); size_t bytes_to_copy = this->byte_size(from_end - from_begin); @@ -778,14 +778,13 @@ void swap(PODArray & lhs, P /// Prevent implicit template instantiation of PODArray for common numeric types -extern template class PODArray, 15, 16>; -extern template class PODArray, 15, 16>; -extern template class PODArray, 15, 16>; -extern template class PODArray, 15, 16>; - -extern template class PODArray, 15, 16>; -extern template class PODArray, 15, 16>; -extern template class PODArray, 15, 16>; -extern template class PODArray, 15, 16>; +extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; +extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; } diff --git a/src/Common/PODArray_fwd.h b/src/Common/PODArray_fwd.h index ec7b3bf4331..56e84d68285 100644 --- a/src/Common/PODArray_fwd.h +++ b/src/Common/PODArray_fwd.h @@ -4,6 +4,7 @@ * PODArray. */ +#include #include #include @@ -22,7 +23,7 @@ class PODArray; /** For columns. Padding is enough to read and write xmm-register at the address of the last element. */ template > -using PaddedPODArray = PODArray; +using PaddedPODArray = PODArray; /** A helper for declaring PODArray that uses inline memory. * The initial size is set to use all the inline bytes, since using less would diff --git a/src/Common/tests/gtest_pod_array.cpp b/src/Common/tests/gtest_pod_array.cpp index 82a6f7589b8..b0c1aab0732 100644 --- a/src/Common/tests/gtest_pod_array.cpp +++ b/src/Common/tests/gtest_pod_array.cpp @@ -532,7 +532,7 @@ TEST(Common, PODNoOverallocation) } } - EXPECT_EQ(capacities, (std::vector{4065, 8161, 16353, 32737, 65505, 131041, 262113, 524257, 1048545})); + EXPECT_EQ(capacities, (std::vector{3969, 8065, 16257, 32641, 65409, 130945, 262017, 524161, 1048449})); } template diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 80efe4f77bf..ae3701c18f0 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -14,17 +14,20 @@ /// The size of the I/O buffer by default. #define DBMS_DEFAULT_BUFFER_SIZE 1048576ULL +#define PADDING_FOR_SIMD 64 + /** Which blocks by default read the data (by number of rows). * Smaller values give better cache locality, less consumption of RAM, but more overhead to process the query. */ -#define DEFAULT_BLOCK_SIZE 65505 /// 65536 minus 16 + 15 bytes padding that we usually have in arrays +#define DEFAULT_BLOCK_SIZE 65409 /// 65536 - PADDING_FOR_SIMD - (PADDING_FOR_SIMD - 1) bytes padding that we usually have in arrays /** Which blocks should be formed for insertion into the table, if we control the formation of blocks. * (Sometimes the blocks are inserted exactly such blocks that have been read / transmitted from the outside, and this parameter does not affect their size.) * More than DEFAULT_BLOCK_SIZE, because in some tables a block of data on the disk is created for each block (quite a big thing), * and if the parts were small, then it would be costly then to combine them. */ -#define DEFAULT_INSERT_BLOCK_SIZE 1048545 /// 1048576 minus 16 + 15 bytes padding that we usually have in arrays +#define DEFAULT_INSERT_BLOCK_SIZE \ + 1048449 /// 1048576 - PADDING_FOR_SIMD - (PADDING_FOR_SIMD - 1) bytes padding that we usually have in arrays /** The same, but for merge operations. Less DEFAULT_BLOCK_SIZE for saving RAM (since all the columns are read). * Significantly less, since there are 10-way mergers. diff --git a/src/Functions/FunctionsRandom.cpp b/src/Functions/FunctionsRandom.cpp index 40522fa1a6e..6078312537f 100644 --- a/src/Functions/FunctionsRandom.cpp +++ b/src/Functions/FunctionsRandom.cpp @@ -124,7 +124,7 @@ void RandImpl::execute(char * output, size_t size) char * end = output + size; constexpr int vec_size = 4; - constexpr int safe_overwrite = 15; + constexpr int safe_overwrite = PADDING_FOR_SIMD - 1; constexpr int bytes_per_write = 4 * sizeof(UInt64x4); UInt64 rand_seed = randomSeed(); diff --git a/src/IO/BufferWithOwnMemory.h b/src/IO/BufferWithOwnMemory.h index 2121747500b..2e451e0032e 100644 --- a/src/IO/BufferWithOwnMemory.h +++ b/src/IO/BufferWithOwnMemory.h @@ -34,8 +34,7 @@ namespace ErrorCodes template > struct Memory : boost::noncopyable, Allocator { - /// Padding is needed to allow usage of 'memcpySmallAllowReadWriteOverflow15' function with this buffer. - static constexpr size_t pad_right = 15; + static constexpr size_t pad_right = PADDING_FOR_SIMD - 1; size_t m_capacity = 0; /// With padding. size_t m_size = 0; diff --git a/src/IO/MMapReadBufferFromFileDescriptor.cpp b/src/IO/MMapReadBufferFromFileDescriptor.cpp index 5a636971fa0..c0eb73f8638 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -28,7 +28,7 @@ void MMapReadBufferFromFileDescriptor::init() BufferBase::set(mapped.getData(), length, 0); size_t page_size = static_cast(::getPageSize()); - ReadBuffer::padded = (length % page_size) > 0 && (length % page_size) <= (page_size - 15); + ReadBuffer::padded = (length % page_size) > 0 && (length % page_size) <= (page_size - (PADDING_FOR_SIMD - 1)); } diff --git a/src/IO/MMapReadBufferFromFileWithCache.cpp b/src/IO/MMapReadBufferFromFileWithCache.cpp index 0d31c29bdaa..503a58b65b9 100644 --- a/src/IO/MMapReadBufferFromFileWithCache.cpp +++ b/src/IO/MMapReadBufferFromFileWithCache.cpp @@ -17,7 +17,7 @@ void MMapReadBufferFromFileWithCache::init() BufferBase::set(mapped->getData(), length, 0); size_t page_size = static_cast(::getPageSize()); - ReadBuffer::padded = (length % page_size) > 0 && (length % page_size) <= (page_size - 15); + ReadBuffer::padded = (length % page_size) > 0 && (length % page_size) <= (page_size - (PADDING_FOR_SIMD - 1)); } diff --git a/src/IO/PeekableReadBuffer.h b/src/IO/PeekableReadBuffer.h index 15283793755..45763863437 100644 --- a/src/IO/PeekableReadBuffer.h +++ b/src/IO/PeekableReadBuffer.h @@ -99,7 +99,7 @@ private: /// creation (for example if PeekableReadBuffer is often created or if we need to remember small amount of /// data after checkpoint), at the beginning we will use small amount of memory on stack and allocate /// larger buffer only if reserved memory is not enough. - char stack_memory[16]; + char stack_memory[PADDING_FOR_SIMD]; bool use_stack_memory = true; }; diff --git a/src/IO/tests/gtest_memory_resize.cpp b/src/IO/tests/gtest_memory_resize.cpp index 8619419a47a..d760a948075 100644 --- a/src/IO/tests/gtest_memory_resize.cpp +++ b/src/IO/tests/gtest_memory_resize.cpp @@ -79,24 +79,24 @@ TEST(MemoryResizeTest, SmallInitAndSmallResize) memory.resize(1); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 1); } { auto memory = Memory(1); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 1); memory.resize(0); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 0); memory.resize(1); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 1); } } @@ -116,52 +116,52 @@ TEST(MemoryResizeTest, SmallInitAndBigResizeOverflowWhenPadding) memory.resize(1); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 1); memory.resize(2); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 17); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD + 1); ASSERT_EQ(memory.m_size, 2); EXPECT_THROW_ERROR_CODE(memory.resize(std::numeric_limits::max()), Exception, ErrorCodes::ARGUMENT_OUT_OF_BOUND); ASSERT_TRUE(memory.m_data); // state is intact after exception - ASSERT_EQ(memory.m_capacity, 17); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD + 1); ASSERT_EQ(memory.m_size, 2); - memory.resize(0x8000000000000000ULL-16); + memory.resize(0x8000000000000000ULL - PADDING_FOR_SIMD); ASSERT_TRUE(memory.m_data); ASSERT_EQ(memory.m_capacity, 0x8000000000000000ULL - 1); - ASSERT_EQ(memory.m_size, 0x8000000000000000ULL - 16); + ASSERT_EQ(memory.m_size, 0x8000000000000000ULL - PADDING_FOR_SIMD); #ifndef ABORT_ON_LOGICAL_ERROR - EXPECT_THROW_ERROR_CODE(memory.resize(0x8000000000000000ULL-15), Exception, ErrorCodes::LOGICAL_ERROR); + EXPECT_THROW_ERROR_CODE(memory.resize(0x8000000000000000ULL - (PADDING_FOR_SIMD - 1)), Exception, ErrorCodes::LOGICAL_ERROR); ASSERT_TRUE(memory.m_data); // state is intact after exception ASSERT_EQ(memory.m_capacity, 0x8000000000000000ULL - 1); - ASSERT_EQ(memory.m_size, 0x8000000000000000ULL - 16); + ASSERT_EQ(memory.m_size, 0x8000000000000000ULL - PADDING_FOR_SIMD); #endif } { auto memory = Memory(1); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 1); EXPECT_THROW_ERROR_CODE(memory.resize(std::numeric_limits::max()), Exception, ErrorCodes::ARGUMENT_OUT_OF_BOUND); ASSERT_TRUE(memory.m_data); // state is intact after exception - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 1); memory.resize(1); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 1); #ifndef ABORT_ON_LOGICAL_ERROR - EXPECT_THROW_ERROR_CODE(memory.resize(0x8000000000000000ULL-15), Exception, ErrorCodes::LOGICAL_ERROR); + EXPECT_THROW_ERROR_CODE(memory.resize(0x8000000000000000ULL - (PADDING_FOR_SIMD - 1)), Exception, ErrorCodes::LOGICAL_ERROR); ASSERT_TRUE(memory.m_data); // state is intact after exception - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 1); #endif } @@ -201,7 +201,7 @@ TEST(MemoryResizeTest, BigInitAndSmallResizeOverflowWhenPadding) { EXPECT_THROW_ERROR_CODE( { - auto memory = Memory(std::numeric_limits::max() - 15); + auto memory = Memory(std::numeric_limits::max() - (PADDING_FOR_SIMD - 1)); } , Exception , ErrorCodes::LOGICAL_ERROR); @@ -210,7 +210,7 @@ TEST(MemoryResizeTest, BigInitAndSmallResizeOverflowWhenPadding) { EXPECT_THROW_ERROR_CODE( { - auto memory = Memory(0x8000000000000000ULL - 15); + auto memory = Memory(0x8000000000000000ULL - (PADDING_FOR_SIMD - 1)); } , Exception , ErrorCodes::LOGICAL_ERROR); @@ -218,10 +218,10 @@ TEST(MemoryResizeTest, BigInitAndSmallResizeOverflowWhenPadding) #endif { - auto memory = Memory(0x8000000000000000ULL - 16); - ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 0x8000000000000000ULL - 1); - ASSERT_EQ(memory.m_size, 0x8000000000000000ULL - 16); + auto memory = Memory(0x8000000000000000ULL - PADDING_FOR_SIMD); + ASSERT_TRUE(memory.m_data); + ASSERT_EQ(memory.m_capacity, 0x8000000000000000ULL - 1); + ASSERT_EQ(memory.m_size, 0x8000000000000000ULL - PADDING_FOR_SIMD); memory.resize(1); ASSERT_TRUE(memory.m_data); @@ -240,32 +240,32 @@ TEST(MemoryResizeTest, AlignmentWithRealAllocator) memory.resize(1); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 1); memory.resize(2); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 17); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD + 1); ASSERT_EQ(memory.m_size, 2); memory.resize(3); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 18); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD + 2); ASSERT_EQ(memory.m_size, 3); memory.resize(4); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 19); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD + 3); ASSERT_EQ(memory.m_size, 4); memory.resize(0); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 19); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD + 3); ASSERT_EQ(memory.m_size, 0); memory.resize(1); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 19); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD + 3); ASSERT_EQ(memory.m_size, 1); } @@ -291,12 +291,12 @@ TEST(MemoryResizeTest, AlignmentWithRealAllocator) memory.resize(1); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 1); memory.resize(32); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 47); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD + 31); ASSERT_EQ(memory.m_size, 32); } } @@ -316,13 +316,12 @@ TEST(MemoryResizeTest, SomeAlignmentOverflowWhenAlignment) memory.resize(1); ASSERT_TRUE(memory.m_data); - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 1); EXPECT_THROW_ERROR_CODE(memory.resize(std::numeric_limits::max()), Exception, ErrorCodes::ARGUMENT_OUT_OF_BOUND); ASSERT_TRUE(memory.m_data); // state is intact after exception - ASSERT_EQ(memory.m_capacity, 16); + ASSERT_EQ(memory.m_capacity, PADDING_FOR_SIMD); ASSERT_EQ(memory.m_size, 1); } - } diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 52c583973d0..a3f2650eac7 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -387,7 +387,7 @@ progress { , stats { rows: 8 blocks: 4 - allocated_bytes: 324 + allocated_bytes: 1092 applied_limit: true rows_before_limit: 8 } diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference index dba46e48e43..58f8b7abfb3 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference @@ -35,7 +35,7 @@ Check total_bytes/total_rows for StripeLog 113 1 Check total_bytes/total_rows for Memory 0 0 -64 1 +256 1 Check total_bytes/total_rows for Buffer 0 0 256 50 diff --git a/tests/queries/0_stateless/02136_scalar_progress.reference b/tests/queries/0_stateless/02136_scalar_progress.reference index e9204f2d02e..5bb4f5640a8 100644 --- a/tests/queries/0_stateless/02136_scalar_progress.reference +++ b/tests/queries/0_stateless/02136_scalar_progress.reference @@ -2,5 +2,5 @@ < X-ClickHouse-Progress: {"read_rows":"65505","read_bytes":"524040","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"} < X-ClickHouse-Progress: {"read_rows":"131010","read_bytes":"1048080","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"} < X-ClickHouse-Progress: {"read_rows":"131011","read_bytes":"1048081","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"} -< X-ClickHouse-Progress: {"read_rows":"131011","read_bytes":"1048081","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"1","result_bytes":"80"} -< X-ClickHouse-Summary: {"read_rows":"131011","read_bytes":"1048081","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"1","result_bytes":"80"} +< X-ClickHouse-Progress: {"read_rows":"131011","read_bytes":"1048081","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"1","result_bytes":"272"} +< X-ClickHouse-Summary: {"read_rows":"131011","read_bytes":"1048081","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"1","result_bytes":"272"} diff --git a/tests/queries/0_stateless/02373_progress_contain_result.reference b/tests/queries/0_stateless/02373_progress_contain_result.reference index 1e7492e2829..a125646e7b8 100644 --- a/tests/queries/0_stateless/02373_progress_contain_result.reference +++ b/tests/queries/0_stateless/02373_progress_contain_result.reference @@ -1 +1 @@ -< X-ClickHouse-Summary: {"read_rows":"100","read_bytes":"800","written_rows":"0","written_bytes":"0","total_rows_to_read":"100","result_rows":"100","result_bytes":"131"} +< X-ClickHouse-Summary: {"read_rows":"100","read_bytes":"800","written_rows":"0","written_bytes":"0","total_rows_to_read":"100","result_rows":"100","result_bytes":"227"} diff --git a/tests/queries/0_stateless/02458_default_setting.reference b/tests/queries/0_stateless/02458_default_setting.reference index 376553843ac..8f4532f370b 100644 --- a/tests/queries/0_stateless/02458_default_setting.reference +++ b/tests/queries/0_stateless/02458_default_setting.reference @@ -1,5 +1,5 @@ -1048545 +1048449 100000 1 -1048545 +1048449 0 From 7626406b86927f7d4643a96a1c9277765307bdb0 Mon Sep 17 00:00:00 2001 From: peter279k Date: Tue, 25 Oct 2022 16:16:59 +0800 Subject: [PATCH 185/188] Add two troubleshootings about installation --- docs/en/operations/troubleshooting.md | 27 +++++++++++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/docs/en/operations/troubleshooting.md b/docs/en/operations/troubleshooting.md index 93bd56087a2..6a1ca3176ad 100644 --- a/docs/en/operations/troubleshooting.md +++ b/docs/en/operations/troubleshooting.md @@ -17,6 +17,33 @@ title: Troubleshooting - Check firewall settings. - If you cannot access the repository for any reason, download packages as described in the [install guide](../getting-started/install.md) article and install them manually using the `sudo dpkg -i ` command. You will also need the `tzdata` package. +### You Cannot Update Deb Packages from ClickHouse Repository with Apt-get {#you-cannot-update-deb-packages-from-clickhouse-repository-with-apt-get} + +- The issue may be happened when the GPG key is changed. + +Please use the following scripts to resolve the issue: + +```bash +sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754 +sudo apt-get update +``` + +### You Get the Unsupported Architecture Warning with Apt-get {#you-get-the-unsupported-architecture-warning-with-apt-get} + +- The completed warning message is as follows: + +``` +N: Skipping acquire of configured file 'main/binary-i386/Packages' as repository 'https://packages.clickhouse.com/deb stable InRelease' doesn't support architecture 'i386' +``` + +To resolve the above issue, please use the following script: + +```bash +sudo rm /var/lib/apt/lists/packages.clickhouse.com_* /var/lib/dpkg/arch +sudo apt-get clean +sudo apt-get autoclean +``` + ## Connecting to the Server {#troubleshooting-accepts-no-connections} Possible issues: From 2c902bbc650783bcd6fdcf0d4cb8cc6986055887 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Tue, 25 Oct 2022 15:57:50 -0400 Subject: [PATCH 186/188] Implement support for different UUID binary formats (#42108) * Implement support for different UUID binary formats * Declare error codes that the implementation uses * Make single-argument constructor explicit * Modernize parts of the solution * Make improvements based on review comments * Declare an error code being used --- .../functions/encoding-functions.md | 8 - .../sql-reference/functions/uuid-functions.md | 46 ++++- src/Functions/FunctionsCodingUUID.cpp | 189 +++++++++++++----- src/IO/WriteHelpers.cpp | 13 -- src/IO/WriteHelpers.h | 3 - .../queries/0_stateless/00396_uuid.reference | 5 + tests/queries/0_stateless/00396_uuid.sql | 6 + 7 files changed, 196 insertions(+), 74 deletions(-) diff --git a/docs/en/sql-reference/functions/encoding-functions.md b/docs/en/sql-reference/functions/encoding-functions.md index eb357df19db..4a6e46e1759 100644 --- a/docs/en/sql-reference/functions/encoding-functions.md +++ b/docs/en/sql-reference/functions/encoding-functions.md @@ -376,14 +376,6 @@ Result: └─────┘ ``` -## UUIDStringToNum(str) - -Accepts a string containing 36 characters in the format `123e4567-e89b-12d3-a456-426655440000`, and returns it as a set of bytes in a FixedString(16). - -## UUIDNumToString(str) - -Accepts a FixedString(16) value. Returns a string containing 36 characters in text format. - ## bitmaskToList(num) Accepts an integer. Returns a string containing the list of powers of two that total the source number when summed. They are comma-separated without spaces in text format, in ascending order. diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index b8f222c2e4e..43542367cd5 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -211,12 +211,19 @@ SELECT toUUIDOrZero('61f0c404-5cb3-11e7-907b-a6006ad3dba0T') AS uuid ## UUIDStringToNum -Accepts a string containing 36 characters in the format `xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, and returns it as a set of bytes in a [FixedString(16)](../../sql-reference/data-types/fixedstring.md). +Accepts `string` containing 36 characters in the format `xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, and returns a [FixedString(16)](../../sql-reference/data-types/fixedstring.md) as its binary representation, with its format optionally specified by `variant` (`Big-endian` by default). + +**Syntax** ``` sql -UUIDStringToNum(String) +UUIDStringToNum(string[, variant = 1]) ``` +**Arguments** + +- `string` — String of 36 characters or FixedString(36). [String](../../sql-reference/syntax.md#syntax-string-literal). +- `variant` — Integer, representing a variant as specified by [RFC4122](https://datatracker.ietf.org/doc/html/rfc4122#section-4.1.1). 1 = `Big-endian` (default), 2 = `Microsoft`. + **Returned value** FixedString(16) @@ -235,14 +242,33 @@ SELECT └──────────────────────────────────────┴──────────────────┘ ``` +``` sql +SELECT + '612f3c40-5d3b-217e-707b-6a546a3d7b29' AS uuid, + UUIDStringToNum(uuid, 2) AS bytes +``` + +``` text +┌─uuid─────────────────────────────────┬─bytes────────────┐ +│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ @ #include -namespace DB -{ +#include -namespace ErrorCodes +namespace DB::ErrorCodes { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_COLUMN; +extern const int ARGUMENT_OUT_OF_BOUND; +extern const int ILLEGAL_COLUMN; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int LOGICAL_ERROR; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ +enum class Representation +{ + BigEndian, + LittleEndian +}; + +std::pair determineBinaryStartIndexWithIncrement(const ptrdiff_t num_bytes, const Representation representation) +{ + if (representation == Representation::BigEndian) + return {0, 1}; + else if (representation == Representation::LittleEndian) + return {num_bytes - 1, -1}; + + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "{} is not handled yet", magic_enum::enum_name(representation)); +} + +void formatHex(const std::span src, UInt8 * dst, const Representation representation) +{ + const auto src_size = std::ssize(src); + const auto [src_start_index, src_increment] = determineBinaryStartIndexWithIncrement(src_size, representation); + for (int src_pos = src_start_index, dst_pos = 0; src_pos >= 0 && src_pos < src_size; src_pos += src_increment, dst_pos += 2) + writeHexByteLowercase(src[src_pos], dst + dst_pos); +} + +void parseHex(const UInt8 * __restrict src, const std::span dst, const Representation representation) +{ + const auto dst_size = std::ssize(dst); + const auto [dst_start_index, dst_increment] = determineBinaryStartIndexWithIncrement(dst_size, representation); + const auto * src_as_char = reinterpret_cast(src); + for (auto dst_pos = dst_start_index, src_pos = 0; dst_pos >= 0 && dst_pos < dst_size; dst_pos += dst_increment, src_pos += 2) + dst[dst_pos] = unhex2(src_as_char + src_pos); +} + +class UUIDSerializer +{ +public: + enum class Variant + { + Default = 1, + Microsoft = 2 + }; + + explicit UUIDSerializer(const Variant variant) + : first_half_binary_representation(variant == Variant::Microsoft ? Representation::LittleEndian : Representation::BigEndian) + { + if (variant != Variant::Default && variant != Variant::Microsoft) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "{} is not handled yet", magic_enum::enum_name(variant)); + } + + void deserialize(const UInt8 * src16, UInt8 * dst36) const + { + formatHex({src16, 4}, &dst36[0], first_half_binary_representation); + dst36[8] = '-'; + formatHex({src16 + 4, 2}, &dst36[9], first_half_binary_representation); + dst36[13] = '-'; + formatHex({src16 + 6, 2}, &dst36[14], first_half_binary_representation); + dst36[18] = '-'; + formatHex({src16 + 8, 2}, &dst36[19], Representation::BigEndian); + dst36[23] = '-'; + formatHex({src16 + 10, 6}, &dst36[24], Representation::BigEndian); + } + + void serialize(const UInt8 * src36, UInt8 * dst16) const + { + /// If string is not like UUID - implementation specific behaviour. + parseHex(&src36[0], {dst16 + 0, 4}, first_half_binary_representation); + parseHex(&src36[9], {dst16 + 4, 2}, first_half_binary_representation); + parseHex(&src36[14], {dst16 + 6, 2}, first_half_binary_representation); + parseHex(&src36[19], {dst16 + 8, 2}, Representation::BigEndian); + parseHex(&src36[24], {dst16 + 10, 6}, Representation::BigEndian); + } + +private: + Representation first_half_binary_representation; +}; + +void checkArgumentCount(const DB::DataTypes & arguments, const std::string_view function_name) +{ + if (const auto argument_count = std::ssize(arguments); argument_count < 1 || argument_count > 2) + throw DB::Exception( + DB::ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1 or 2", + function_name, + argument_count); +} + +void checkFormatArgument(const DB::DataTypes & arguments, const std::string_view function_name) +{ + if (const auto argument_count = std::ssize(arguments); + argument_count > 1 && !DB::WhichDataType(arguments[1]).isInt8() && !DB::WhichDataType(arguments[1]).isUInt8()) + throw DB::Exception( + DB::ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, expected Int8 or UInt8 type", + arguments[1]->getName(), + function_name); +} + +UUIDSerializer::Variant parseVariant(const DB::ColumnsWithTypeAndName & arguments) +{ + if (arguments.size() < 2) + return UUIDSerializer::Variant::Default; + + const auto representation = static_cast>(arguments[1].column->getInt(0)); + const auto as_enum = magic_enum::enum_cast(representation); + if (!as_enum) + throw DB::Exception(DB::ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Expected UUID variant, got {}", representation); + + return *as_enum; +} +} + +namespace DB +{ constexpr size_t uuid_bytes_length = 16; constexpr size_t uuid_text_length = 36; class FunctionUUIDNumToString : public IFunction { - public: static constexpr auto name = "UUIDNumToString"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override { return 1; } + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } bool isInjective(const ColumnsWithTypeAndName &) const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isVariadic() const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { + checkArgumentCount(arguments, name); + const auto * ptr = checkAndGetDataType(arguments[0].get()); if (!ptr || ptr->getN() != uuid_bytes_length) throw Exception("Illegal type " + arguments[0]->getName() + @@ -50,6 +165,8 @@ public: ", expected FixedString(" + toString(uuid_bytes_length) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + checkFormatArgument(arguments, name); + return std::make_shared(); } @@ -59,7 +176,7 @@ public: { const ColumnWithTypeAndName & col_type_name = arguments[0]; const ColumnPtr & column = col_type_name.column; - + const auto variant = parseVariant(arguments); if (const auto * col_in = checkAndGetColumn(column.get())) { if (col_in->getN() != uuid_bytes_length) @@ -82,9 +199,10 @@ public: size_t src_offset = 0; size_t dst_offset = 0; + const UUIDSerializer uuid_serializer(variant); for (size_t i = 0; i < size; ++i) { - formatUUID(&vec_in[src_offset], &vec_res[dst_offset]); + uuid_serializer.deserialize(&vec_in[src_offset], &vec_res[dst_offset]); src_offset += uuid_bytes_length; dst_offset += uuid_text_length; vec_res[dst_offset] = 0; @@ -104,55 +222,33 @@ public: class FunctionUUIDStringToNum : public IFunction { -private: - static void parseHex(const UInt8 * __restrict src, UInt8 * __restrict dst, const size_t num_bytes) - { - size_t src_pos = 0; - size_t dst_pos = 0; - for (; dst_pos < num_bytes; ++dst_pos) - { - dst[dst_pos] = unhex2(reinterpret_cast(&src[src_pos])); - src_pos += 2; - } - } - - static void parseUUID(const UInt8 * src36, UInt8 * dst16) - { - /// If string is not like UUID - implementation specific behaviour. - - parseHex(&src36[0], &dst16[0], 4); - parseHex(&src36[9], &dst16[4], 2); - parseHex(&src36[14], &dst16[6], 2); - parseHex(&src36[19], &dst16[8], 2); - parseHex(&src36[24], &dst16[10], 6); - } - public: static constexpr auto name = "UUIDStringToNum"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override { return 1; } + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } bool isInjective(const ColumnsWithTypeAndName &) const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isVariadic() const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { + checkArgumentCount(arguments, name); + /// String or FixedString(36) if (!isString(arguments[0])) { const auto * ptr = checkAndGetDataType(arguments[0].get()); if (!ptr || ptr->getN() != uuid_text_length) throw Exception("Illegal type " + arguments[0]->getName() + - " of argument of function " + getName() + + " of first argument of function " + getName() + ", expected FixedString(" + toString(uuid_text_length) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } + checkFormatArgument(arguments, name); + return std::make_shared(uuid_bytes_length); } @@ -163,6 +259,7 @@ public: const ColumnWithTypeAndName & col_type_name = arguments[0]; const ColumnPtr & column = col_type_name.column; + const UUIDSerializer uuid_serializer(parseVariant(arguments)); if (const auto * col_in = checkAndGetColumn(column.get())) { const auto & vec_in = col_in->getChars(); @@ -184,7 +281,7 @@ public: size_t string_size = offsets_in[i] - src_offset; if (string_size == uuid_text_length + 1) - parseUUID(&vec_in[src_offset], &vec_res[dst_offset]); + uuid_serializer.serialize(&vec_in[src_offset], &vec_res[dst_offset]); else memset(&vec_res[dst_offset], 0, uuid_bytes_length); @@ -216,7 +313,7 @@ public: for (size_t i = 0; i < size; ++i) { - parseUUID(&vec_in[src_offset], &vec_res[dst_offset]); + uuid_serializer.serialize(&vec_in[src_offset], &vec_res[dst_offset]); src_offset += uuid_text_length; dst_offset += uuid_bytes_length; } diff --git a/src/IO/WriteHelpers.cpp b/src/IO/WriteHelpers.cpp index cb341e60a8b..a9788505995 100644 --- a/src/IO/WriteHelpers.cpp +++ b/src/IO/WriteHelpers.cpp @@ -18,19 +18,6 @@ void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes) } } -void formatUUID(const UInt8 * src16, UInt8 * dst36) -{ - formatHex(&src16[0], &dst36[0], 4); - dst36[8] = '-'; - formatHex(&src16[4], &dst36[9], 2); - dst36[13] = '-'; - formatHex(&src16[6], &dst36[14], 2); - dst36[18] = '-'; - formatHex(&src16[8], &dst36[19], 2); - dst36[23] = '-'; - formatHex(&src16[10], &dst36[24], 6); -} - /** Function used when byte ordering is important when parsing uuid * ex: When we create an UUID type */ diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 42d84e080af..39024b33eb1 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -624,9 +624,6 @@ inline void writeXMLStringForTextElement(std::string_view s, WriteBuffer & buf) writeXMLStringForTextElement(s.data(), s.data() + s.size(), buf); } -template -void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes); -void formatUUID(const UInt8 * src16, UInt8 * dst36); void formatUUID(std::reverse_iterator src16, UInt8 * dst36); inline void writeUUIDText(const UUID & uuid, WriteBuffer & buf) diff --git a/tests/queries/0_stateless/00396_uuid.reference b/tests/queries/0_stateless/00396_uuid.reference index d70322ec4c1..588f11cb466 100644 --- a/tests/queries/0_stateless/00396_uuid.reference +++ b/tests/queries/0_stateless/00396_uuid.reference @@ -6,3 +6,8 @@ 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 3f1ed72e-f7fe-4459-9cbe-95fe9298f845 1 +-- UUID variants -- +00112233445566778899AABBCCDDEEFF +33221100554477668899AABBCCDDEEFF +00112233-4455-6677-8899-aabbccddeeff +00112233-4455-6677-8899-aabbccddeeff diff --git a/tests/queries/0_stateless/00396_uuid.sql b/tests/queries/0_stateless/00396_uuid.sql index 9d8b48bddb0..4ad659e2464 100644 --- a/tests/queries/0_stateless/00396_uuid.sql +++ b/tests/queries/0_stateless/00396_uuid.sql @@ -11,3 +11,9 @@ with generateUUIDv4() as uuid, identity(lower(hex(reverse(reinterpretAsString(uuid))))) as str, reinterpretAsUUID(reverse(unhex(str))) as uuid2 select uuid = uuid2; + +select '-- UUID variants --'; +select hex(UUIDStringToNum('00112233-4455-6677-8899-aabbccddeeff', 1)); +select hex(UUIDStringToNum('00112233-4455-6677-8899-aabbccddeeff', 2)); +select UUIDNumToString(UUIDStringToNum('00112233-4455-6677-8899-aabbccddeeff', 1), 1); +select UUIDNumToString(UUIDStringToNum('00112233-4455-6677-8899-aabbccddeeff', 2), 2); From 97e4d5a7d826fca0a5a0dcb7bf67034efef1a60f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Oct 2022 06:19:39 +0300 Subject: [PATCH 187/188] Update CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index c2def2f787e..8b97622dcd1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -33,6 +33,7 @@ * Add support for methods `lz4`, `bz2`, `snappy` in HTTP's `Accept-Encoding` which is a non-standard extension to HTTP protocol. [#42071](https://github.com/ClickHouse/ClickHouse/pull/42071) ([Nikolay Degterinsky](https://github.com/evillique)). #### Experimental Feature +* Added new infrastructure for query analysis and planning under the `allow_experimental_analyzer` setting. [#31796](https://github.com/ClickHouse/ClickHouse/pull/31796) ([Maksim Kita](https://github.com/kitaisreal)). * Initial implementation of Kusto Query Language. Please don't use it. [#37961](https://github.com/ClickHouse/ClickHouse/pull/37961) ([Yong Wang](https://github.com/kashwy)). #### Performance Improvement From cca0d0a4b8564c5034b1914e7d22dac8b0ee10b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Oct 2022 06:24:57 +0300 Subject: [PATCH 188/188] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8b97622dcd1..22f6afc4901 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,7 +11,7 @@ **[ClickHouse release v22.1, 2022-01-18](#221)**
**[Changelog for 2021](https://clickhouse.com/docs/en/whats-new/changelog/2021/)**
-### ClickHouse release 22.10, 2022-10-25 +### ClickHouse release 22.10, 2022-10-26 #### Backward Incompatible Change * Rename cache commands: `show caches` -> `show filesystem caches`, `describe cache` -> `describe filesystem cache`. [#41508](https://github.com/ClickHouse/ClickHouse/pull/41508) ([Kseniia Sumarokova](https://github.com/kssenii)).