From dbf6e6c85e098826604edcfc2b9b73a9f8883c1b Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 4 Sep 2024 16:05:13 +0800 Subject: [PATCH] review fix --- src/Core/Settings.h | 5 ++-- src/Core/SettingsChangesHistory.cpp | 5 ++-- src/Interpreters/HashJoin/AddedColumns.h | 8 +++--- src/Interpreters/HashJoin/HashJoin.cpp | 26 +++++++++++++------ src/Interpreters/TableJoin.cpp | 1 + src/Interpreters/TableJoin.h | 2 ++ tests/performance/all_join_opt.xml | 4 +-- .../experimental_settings_ignore.txt | 1 + 8 files changed, 34 insertions(+), 18 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c1433ca7250..386fd4e74ee 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -922,8 +922,9 @@ class IColumn; M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \ M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \ M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \ - M(Int32, join_to_sort_perkey_rows_threshold, 40, "The lower limit of per-key average rows in the right table to determine whether to sort it in hash join.", 0) \ - M(Int32, join_to_sort_table_rows_threshold, 10000, "The upper limit of rows in the right table to determine whether to sort it in hash join.", 0) \ + M(Int32, join_to_sort_perkey_rows_threshold, 40, "Rerange the right table by key in hash join when the per-key average rows of it exceed this value (means the table keys is dense) and its number of rows is not too many(controlled by `join_to_sort_table_rows_threshold`), to make the join output by the key's batch, which would improve performance.", 0) \ + M(Int32, join_to_sort_table_rows_threshold, 10000, "Rerange the right table by key in hash join when its number of rows not exceed this value and the table keys is dense (controlled by `join_to_sort_perkey_rows_threshold`), to make the join performance improve as output by the key's batch, but not cost too much on the table reranging.", 0) \ + M(Bool, allow_experimental_inner_join_right_table_sorting, false, "If it is set to true, and the conditions of `join_to_sort_perkey_rows_threshold` and `join_to_sort_perkey_rows_threshold` are met, then we will try to rerange the right table by key to improve the performance in hash join.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 392f0dbc2ee..2a38af85b08 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -95,8 +95,9 @@ static std::initializer_listavgPerKeyRows()) + , output_by_row_list_threshold(join.getTableJoin().outputByRowListPerkeyRowsThreshold()) + , join_data_sorted(join.getJoinedData()->sorted) , is_join_get(is_join_get_) { size_t num_columns_to_add = block_with_columns_to_add.columns(); @@ -113,9 +116,6 @@ public: if (columns[j]->isNullable() && !saved_column->isNullable()) nullable_column_ptrs[j] = typeid_cast(columns[j].get()); } - join_data_avg_perkey_rows = join.getJoinedData()->avgPerKeyRows(); - output_by_row_list_threshold = join.getTableJoin().outputByRowListPerkeyRowsThreshold(); - join_data_sorted = join.getJoinedData()->sorted; } size_t size() const { return columns.size(); } @@ -148,9 +148,9 @@ public: std::unique_ptr offsets_to_replicate; bool need_filter = false; bool output_by_row_list = false; - bool join_data_sorted = false; size_t join_data_avg_perkey_rows = 0; size_t output_by_row_list_threshold = 0; + bool join_data_sorted = false; IColumn::Filter filter; void reserve(bool need_replicate) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 59888d7a71d..5b095368131 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -1364,14 +1364,16 @@ template void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) { constexpr JoinFeatures join_features; - if constexpr (join_features.is_all_join && (join_features.left || join_features.inner)) + if constexpr (!join_features.is_all_join || (!join_features.left && !join_features.inner)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Only left or inner join table can be reranged."); + else { auto merge_rows_into_one_block = [&](BlocksList & blocks, RowRefList & rows_ref) { auto it = rows_ref.begin(); if (it.ok()) { - if (blocks.empty() || blocks.back().rows() > DEFAULT_BLOCK_SIZE) + if (blocks.empty() || blocks.back().rows() >= DEFAULT_BLOCK_SIZE) blocks.emplace_back(it->block->cloneEmpty()); } else @@ -1384,7 +1386,7 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) { for (size_t i = 0; i < block.columns(); ++i) { - auto & col = *(block.getByPosition(i).column->assumeMutable()); + auto & col = block.getByPosition(i).column->assumeMutableRef(); col.insertFrom(*it->block->getByPosition(i).column, it->row_num); } } @@ -1419,26 +1421,34 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) void HashJoin::tryRerangeRightTableData() { - if ((kind != JoinKind::Inner && kind != JoinKind::Left) || strictness != JoinStrictness::All || table_join->getMixedJoinExpression()) + if (!table_join->allowJoinSorting() || table_join->getMixedJoinExpression() || !isInnerOrLeft(kind) || strictness != JoinStrictness::All) return; + /// We should not rerange the right table on such conditions: + /// 1. the right table is already reranged by key or it is empty. + /// 2. the join clauses size is greater than 1, like `...join on a.key1=b.key1 or a.key2=b.key2`, we can not rerange the right table on different set of keys. + /// 3. the number of right table rows exceed the threshold, which may result in a significant cost for reranging and lead to performance degradation. + /// 4. the keys of right table is very sparse, which may result in insignificant performance improvement after reranging by key. if (!data || data->sorted || data->blocks.empty() || data->maps.size() > 1 || data->rows_to_join > table_join->sortRightTableRowsThreshold() || data->avgPerKeyRows() < table_join->sortRightPerkeyRowsThreshold()) return; if (data->keys_to_join == 0) data->keys_to_join = getTotalRowCount(); + + /// If the there is no columns to add, means no columns to output, then the rerange would not improve performance by using column's `insertRangeFrom` + /// to replace column's `insertFrom` to make the output. if (sample_block_with_columns_to_add.columns() == 0) { - LOG_DEBUG(log, "The joined right table total rows :{}, total keys :{}, columns added:{}", - data->rows_to_join, data->keys_to_join, sample_block_with_columns_to_add.columns()); + LOG_DEBUG(log, "The joined right table total rows :{}, total keys :{}", data->rows_to_join, data->keys_to_join); return; } - joinDispatch( + [[maybe_unused]] bool result = joinDispatch( kind, strictness, data->maps.front(), - false, + /*prefer_use_maps_all*/ false, [&](auto kind_, auto strictness_, auto & map_) { tryRerangeRightTableDataImpl(map_); }); + chassert(result); data->sorted = true; } diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 8bcaef77939..8d79b88190b 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -118,6 +118,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , output_by_rowlist_perkey_rows_threshold(settings.join_output_by_rowlist_perkey_rows_threshold) , sort_right_perkey_rows_threshold(settings.join_to_sort_perkey_rows_threshold) , sort_right_table_rows_threshold(settings.join_to_sort_table_rows_threshold) + , allow_join_sorting(settings.allow_experimental_inner_join_right_table_sorting) , max_memory_usage(settings.max_memory_usage) , tmp_volume(tmp_volume_) , tmp_data(tmp_data_) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 09d7f0f2b2a..c7926271a67 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -151,6 +151,7 @@ private: const size_t output_by_rowlist_perkey_rows_threshold = 0; const size_t sort_right_perkey_rows_threshold = 0; const size_t sort_right_table_rows_threshold = 0; + const bool allow_join_sorting = false; /// Value if setting max_memory_usage for query, can be used when max_bytes_in_join is not specified. size_t max_memory_usage = 0; @@ -301,6 +302,7 @@ public: size_t outputByRowListPerkeyRowsThreshold() const { return output_by_rowlist_perkey_rows_threshold; } size_t sortRightPerkeyRowsThreshold() const { return sort_right_perkey_rows_threshold; } size_t sortRightTableRowsThreshold() const { return sort_right_table_rows_threshold; } + bool allowJoinSorting() const { return allow_join_sorting; } 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; } diff --git a/tests/performance/all_join_opt.xml b/tests/performance/all_join_opt.xml index 0ab9c39f67c..ed8805a2e5f 100644 --- a/tests/performance/all_join_opt.xml +++ b/tests/performance/all_join_opt.xml @@ -5,8 +5,8 @@ INSERT INTO test SELECT number % 10000, number % 10000, number % 10000 FROM numbers(10000000) INSERT INTO test1 SELECT number % 1000 , number % 1000, number % 1000 FROM numbers(100000) - SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b - SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b + SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b SETTINGS allow_experimental_inner_join_right_table_sorting=true, join_to_sort_table_rows_threshold=100000 + SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b SETTINGS allow_experimental_inner_join_right_table_sorting=true, join_to_sort_table_rows_threshold=100000 SELECT MAX(test1.a) FROM test RIGHT JOIN test1 on test.b = test1.b SELECT MAX(test1.a) FROM test FULL JOIN test1 on test.b = test1.b diff --git a/utils/check-style/experimental_settings_ignore.txt b/utils/check-style/experimental_settings_ignore.txt index 94c46cf562e..358374494a3 100644 --- a/utils/check-style/experimental_settings_ignore.txt +++ b/utils/check-style/experimental_settings_ignore.txt @@ -31,6 +31,7 @@ allow_experimental_statistics allow_experimental_time_series_table allow_experimental_undrop_table_query allow_experimental_usearch_index +allow_experimental_inner_join_right_table_sorting allow_get_client_http_header allow_introspection_functions allow_materialized_view_with_bad_select