From 4f33c23b5215e05bf54918d6d4ab98c046b25143 Mon Sep 17 00:00:00 2001 From: Nitin Kashyap Date: Mon, 27 Nov 2023 13:20:53 +0530 Subject: [PATCH 1/3] [feature](datalake) Add BucketShuffleJoin support for Hive table data generated by Spark. (27783) 1. Original planner updated to consider BucketShuffle for bucketed hive table 2. Neerids planner updated for bucketShuffle join on hive tables. 3. Added spark style hash calculation in BE for shuffle on one side. 4. Added shuffle hash selection based on left(non-shuffling) side. --- be/src/util/hash_util.hpp | 8 + be/src/vec/columns/column.h | 27 +++ be/src/vec/columns/column_array.cpp | 54 ++++++ be/src/vec/columns/column_array.h | 6 + be/src/vec/columns/column_decimal.cpp | 50 ++++++ be/src/vec/columns/column_decimal.h | 14 +- be/src/vec/columns/column_map.cpp | 54 ++++++ be/src/vec/columns/column_map.h | 6 + be/src/vec/columns/column_nullable.cpp | 38 +++++ be/src/vec/columns/column_nullable.h | 5 + be/src/vec/columns/column_string.cpp | 23 +++ be/src/vec/columns/column_string.h | 23 +++ be/src/vec/columns/column_struct.cpp | 15 ++ be/src/vec/columns/column_struct.h | 6 + be/src/vec/columns/column_vector.cpp | 35 ++++ be/src/vec/columns/column_vector.h | 23 +++ be/src/vec/runtime/partitioner.cpp | 28 +++- be/src/vec/runtime/partitioner.h | 18 ++ be/src/vec/sink/vdata_stream_sender.h | 7 + .../catalog/HiveExternalDistributionInfo.java | 95 +++++++++++ .../datasource/FederationBackendPolicy.java | 10 ++ .../doris/datasource/FileQueryScanNode.java | 80 ++++++++- .../datasource/hive/HMSExternalTable.java | 132 ++++++++++++++- .../doris/datasource/hive/HiveBucketUtil.java | 5 +- .../datasource/hive/source/HiveScanNode.java | 40 +++++ .../translator/PhysicalPlanTranslator.java | 30 +++- .../ChildOutputPropertyDeriver.java | 2 +- .../ChildrenPropertiesRegulator.java | 2 +- .../properties/DistributionSpecHash.java | 50 ++++-- .../EnforceMissingPropertiesHelper.java | 4 +- .../LogicalFileScanToPhysicalFileScan.java | 50 +++++- .../apache/doris/planner/DataPartition.java | 23 ++- .../doris/planner/DistributedPlanner.java | 93 ++++++++++- .../apache/doris/planner/OlapScanNode.java | 3 +- .../java/org/apache/doris/qe/Coordinator.java | 55 +++++- .../org/apache/doris/qe/CoordinatorTest.java | 7 +- gensrc/thrift/Ddl.thrift | 6 +- gensrc/thrift/Partitions.thrift | 7 + .../hive/test_hive_spark_clustered_table.out | 158 ++++++++++++++++++ .../test_hive_spark_clustered_table.groovy | 67 ++++++++ .../join/bucket_shuffle_join.groovy | 83 +++++++++ 41 files changed, 1391 insertions(+), 51 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/catalog/HiveExternalDistributionInfo.java create mode 100644 regression-test/data/external_table_p0/hive/test_hive_spark_clustered_table.out create mode 100644 regression-test/suites/external_table_p0/hive/test_hive_spark_clustered_table.groovy create mode 100644 regression-test/suites/nereids_p0/join/bucket_shuffle_join.groovy diff --git a/be/src/util/hash_util.hpp b/be/src/util/hash_util.hpp index fbf10b75ae02c0..633e5c2b90661e 100644 --- a/be/src/util/hash_util.hpp +++ b/be/src/util/hash_util.hpp @@ -125,6 +125,9 @@ class HashUtil { // refer to https://github.com/apache/commons-codec/blob/master/src/main/java/org/apache/commons/codec/digest/MurmurHash3.java static const uint32_t MURMUR3_32_SEED = 104729; + // refer https://github.com/apache/spark/blob/v3.5.0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala#L615 + static const uint32_t SPARK_MURMUR_32_SEED = 42; + // modify from https://github.com/aappleby/smhasher/blob/master/src/MurmurHash3.cpp static uint32_t murmur_hash3_32(const void* key, int64_t len, uint32_t seed) { uint32_t out = 0; @@ -132,6 +135,11 @@ class HashUtil { return out; } + static uint32_t murmur_hash3_32_null(uint32_t seed) { + static const int INT_VALUE = 0; + return murmur_hash3_32((const unsigned char*)(&INT_VALUE), 4, seed); + } + static const int MURMUR_R = 47; // Murmur2 hash implementation returning 64-bit hashes. diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index 729e5470c97ad5..917a8283d5787b 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -57,6 +57,18 @@ class SipHash; } \ } +#define DO_MURMUR_HASHES_FUNCTION_COLUMN_IMPL(SEED) \ + if (null_data == nullptr) { \ + for (size_t i = 0; i < s; i++) { \ + hashes[i] = HashUtil::murmur_hash3_32(&data[i], sizeof(T), SEED); \ + } \ + } else { \ + for (size_t i = 0; i < s; i++) { \ + if (null_data[i] == 0) \ + hashes[i] = HashUtil::murmur_hash3_32(&data[i], sizeof(T), SEED); \ + } \ + } + namespace doris::vectorized { class Arena; @@ -398,6 +410,21 @@ class IColumn : public COW { "Method update_crc_with_value is not supported for " + get_name()); } + /// Update state of murmur3 hash function (spark files) with value of n elements to avoid the virtual + /// function call null_data to mark whether need to do hash compute, null_data == nullptr + /// means all element need to do hash function, else only *null_data != 0 need to do hash func + virtual void update_murmurs_with_value(int32_t* __restrict hash, PrimitiveType type, + int32_t rows, uint32_t offset = 0, + const uint8_t* __restrict null_data = nullptr) const { + LOG(FATAL) << get_name() << "update_murmurs_with_value not supported"; + } + + // use range for one hash value to avoid virtual function call in loop + virtual void update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const { + LOG(FATAL) << get_name() << " update_murmur_with_value not supported"; + } + /** Removes elements that don't match the filter. * Is used in WHERE and HAVING operations. * If result_size_hint > 0, then makes advance reserve(result_size_hint) for the result column; diff --git a/be/src/vec/columns/column_array.cpp b/be/src/vec/columns/column_array.cpp index 0df0ddcb0f30d5..814aea9d6af83f 100644 --- a/be/src/vec/columns/column_array.cpp +++ b/be/src/vec/columns/column_array.cpp @@ -319,6 +319,60 @@ void ColumnArray::update_crcs_with_value(uint32_t* __restrict hash, PrimitiveTyp } } +// for every array row calculate murmurHash +void ColumnArray::update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const { + auto& offsets_column = get_offsets(); + if (hash == 0) { + hash = HashUtil::SPARK_MURMUR_32_SEED; + } + if (null_data) { + for (size_t i = start; i < end; ++i) { + if (null_data[i] == 0) { + size_t elem_size = offsets_column[i] - offsets_column[i - 1]; + if (elem_size == 0) { + hash = HashUtil::murmur_hash3_32(reinterpret_cast(&elem_size), + sizeof(elem_size), hash); + } else { + get_data().update_murmur_with_value(offsets_column[i - 1], offsets_column[i], + hash, nullptr); + } + } + } + } else { + for (size_t i = start; i < end; ++i) { + size_t elem_size = offsets_column[i] - offsets_column[i - 1]; + if (elem_size == 0) { + hash = HashUtil::murmur_hash3_32(reinterpret_cast(&elem_size), + sizeof(elem_size), hash); + } else { + get_data().update_murmur_with_value(offsets_column[i - 1], offsets_column[i], hash, + nullptr); + } + } + } +} + +void ColumnArray::update_murmurs_with_value(int32_t* __restrict hash, PrimitiveType type, + int32_t rows, uint32_t offset, + const uint8_t* __restrict null_data) const { + auto s = rows; + DCHECK(s == size()); + + if (null_data) { + for (size_t i = 0; i < s; ++i) { + // every row + if (null_data[i] == 0) { + update_murmur_with_value(i, i + 1, hash[i], nullptr); + } + } + } else { + for (size_t i = 0; i < s; ++i) { + update_murmur_with_value(i, i + 1, hash[i], nullptr); + } + } +} + void ColumnArray::insert(const Field& x) { if (x.is_null()) { get_data().insert(Null()); diff --git a/be/src/vec/columns/column_array.h b/be/src/vec/columns/column_array.h index 4dbc8e91e52b88..1052b6c70e9fd3 100644 --- a/be/src/vec/columns/column_array.h +++ b/be/src/vec/columns/column_array.h @@ -140,6 +140,8 @@ class ColumnArray final : public COWHelper { const uint8_t* __restrict null_data) const override; void update_crc_with_value(size_t start, size_t end, uint32_t& hash, const uint8_t* __restrict null_data) const override; + void update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const override; void update_hashes_with_value(uint64_t* __restrict hashes, const uint8_t* __restrict null_data = nullptr) const override; @@ -148,6 +150,10 @@ class ColumnArray final : public COWHelper { uint32_t offset = 0, const uint8_t* __restrict null_data = nullptr) const override; + void update_murmurs_with_value(int32_t* __restrict hash, PrimitiveType type, int32_t rows, + uint32_t offset = 0, + const uint8_t* __restrict null_data = nullptr) const override; + void insert_range_from(const IColumn& src, size_t start, size_t length) override; void insert_range_from_ignore_overflow(const IColumn& src, size_t start, size_t length) override; diff --git a/be/src/vec/columns/column_decimal.cpp b/be/src/vec/columns/column_decimal.cpp index cf0193b29e187e..5b45b96afe628e 100644 --- a/be/src/vec/columns/column_decimal.cpp +++ b/be/src/vec/columns/column_decimal.cpp @@ -183,6 +183,56 @@ void ColumnDecimal::update_crcs_with_value(uint32_t* __restrict hashes, Primi } } +template +void ColumnDecimal::update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const { + if (null_data == nullptr) { + for (size_t i = start; i < end; i++) { + if constexpr (!IsDecimalV2) { + hash = HashUtil::murmur_hash3_32(&data[i], sizeof(T), + HashUtil::SPARK_MURMUR_32_SEED); + } else { + decimalv2_do_murmur(i, hash); + } + } + } else { + for (size_t i = start; i < end; i++) { + if (null_data[i] == 0) { + if constexpr (!IsDecimalV2) { + hash = HashUtil::murmur_hash3_32(&data[i], sizeof(T), + HashUtil::SPARK_MURMUR_32_SEED); + } else { + decimalv2_do_murmur(i, hash); + } + } + } + } +} + +template +void ColumnDecimal::update_murmurs_with_value(int32_t* __restrict hashes, PrimitiveType type, + int32_t rows, uint32_t offset, + const uint8_t* __restrict null_data) const { + auto s = rows; + DCHECK(s == size()); + + if constexpr (!IsDecimalV2) { + DO_MURMUR_HASHES_FUNCTION_COLUMN_IMPL(HashUtil::SPARK_MURMUR_32_SEED) + } else { + if (null_data == nullptr) { + for (size_t i = 0; i < s; i++) { + decimalv2_do_murmur(i, hashes[i]); + } + } else { + for (size_t i = 0; i < s; i++) { + if (null_data[i] == 0) { + decimalv2_do_murmur(i, hashes[i]); + } + } + } + } +} + template void ColumnDecimal::update_xxHash_with_value(size_t start, size_t end, uint64_t& hash, const uint8_t* __restrict null_data) const { diff --git a/be/src/vec/columns/column_decimal.h b/be/src/vec/columns/column_decimal.h index dde7a1c6237481..8f91ab709a65a6 100644 --- a/be/src/vec/columns/column_decimal.h +++ b/be/src/vec/columns/column_decimal.h @@ -180,12 +180,16 @@ class ColumnDecimal final : public COWHelper> { void update_crcs_with_value(uint32_t* __restrict hashes, PrimitiveType type, uint32_t rows, uint32_t offset, const uint8_t* __restrict null_data) const override; + void update_murmurs_with_value(int32_t* __restrict hashes, PrimitiveType type, int32_t rows, + uint32_t offset, + const uint8_t* __restrict null_data) const override; void update_xxHash_with_value(size_t start, size_t end, uint64_t& hash, const uint8_t* __restrict null_data) const override; void update_crc_with_value(size_t start, size_t end, uint32_t& hash, const uint8_t* __restrict null_data) const override; - + void update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const override; int compare_at(size_t n, size_t m, const IColumn& rhs_, int nan_direction_hint) const override; void get_permutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation& res) const override; @@ -284,6 +288,14 @@ class ColumnDecimal final : public COWHelper> { hash = HashUtil::zlib_crc_hash(&int_val, sizeof(int_val), hash); hash = HashUtil::zlib_crc_hash(&frac_val, sizeof(frac_val), hash); }; + + void ALWAYS_INLINE decimalv2_do_murmur(size_t i, int32_t& hash) const { + const auto& dec_val = (const DecimalV2Value&)data[i]; + int64_t int_val = dec_val.int_value(); + int32_t frac_val = dec_val.frac_value(); + hash = HashUtil::murmur_hash3_32(&int_val, sizeof(int_val), hash); + hash = HashUtil::murmur_hash3_32(&frac_val, sizeof(frac_val), hash); + }; }; template diff --git a/be/src/vec/columns/column_map.cpp b/be/src/vec/columns/column_map.cpp index eb3b431a229d7b..06017ed0f391a0 100644 --- a/be/src/vec/columns/column_map.cpp +++ b/be/src/vec/columns/column_map.cpp @@ -343,6 +343,40 @@ void ColumnMap::update_crc_with_value(size_t start, size_t end, uint32_t& hash, } } +void ColumnMap::update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const { + auto& offsets = get_offsets(); + if (hash == 0) { + hash = HashUtil::SPARK_MURMUR_32_SEED; + } + if (null_data) { + for (size_t i = start; i < end; ++i) { + if (null_data[i] == 0) { + size_t kv_size = offsets[i] - offsets[i - 1]; + if (kv_size == 0) { + hash = HashUtil::murmur_hash3_32(reinterpret_cast(&kv_size), + sizeof(kv_size), hash); + } else { + get_keys().update_murmur_with_value(offsets[i - 1], offsets[i], hash, nullptr); + get_values().update_murmur_with_value(offsets[i - 1], offsets[i], hash, + nullptr); + } + } + } + } else { + for (size_t i = start; i < end; ++i) { + size_t kv_size = offsets[i] - offsets[i - 1]; + if (kv_size == 0) { + hash = HashUtil::murmur_hash3_32(reinterpret_cast(&kv_size), + sizeof(kv_size), hash); + } else { + get_keys().update_murmur_with_value(offsets[i - 1], offsets[i], hash, nullptr); + get_values().update_murmur_with_value(offsets[i - 1], offsets[i], hash, nullptr); + } + } + } +} + void ColumnMap::update_hashes_with_value(uint64_t* hashes, const uint8_t* null_data) const { size_t s = size(); if (null_data) { @@ -378,6 +412,26 @@ void ColumnMap::update_crcs_with_value(uint32_t* __restrict hash, PrimitiveType } } +void ColumnMap::update_murmurs_with_value(int32_t* __restrict hash, PrimitiveType type, + int32_t rows, uint32_t offset, + const uint8_t* __restrict null_data) const { + auto s = rows; + DCHECK(s == size()); + + if (null_data) { + for (size_t i = 0; i < s; ++i) { + // every row + if (null_data[i] == 0) { + update_murmur_with_value(i, i + 1, hash[i], nullptr); + } + } + } else { + for (size_t i = 0; i < s; ++i) { + update_murmur_with_value(i, i + 1, hash[i], nullptr); + } + } +} + void ColumnMap::insert_range_from(const IColumn& src, size_t start, size_t length) { if (length == 0) { return; diff --git a/be/src/vec/columns/column_map.h b/be/src/vec/columns/column_map.h index ae482a2d4e058c..62ad28eb12061e 100644 --- a/be/src/vec/columns/column_map.h +++ b/be/src/vec/columns/column_map.h @@ -161,6 +161,8 @@ class ColumnMap final : public COWHelper { const uint8_t* __restrict null_data) const override; void update_crc_with_value(size_t start, size_t end, uint32_t& hash, const uint8_t* __restrict null_data) const override; + void update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const override; void update_hashes_with_value(uint64_t* __restrict hashes, const uint8_t* __restrict null_data = nullptr) const override; @@ -169,6 +171,10 @@ class ColumnMap final : public COWHelper { uint32_t offset = 0, const uint8_t* __restrict null_data = nullptr) const override; + void update_murmurs_with_value(int32_t* __restrict hash, PrimitiveType type, int32_t rows, + uint32_t offset = 0, + const uint8_t* __restrict null_data = nullptr) const override; + /******************** keys and values ***************/ const ColumnPtr& get_keys_ptr() const { return keys_column; } ColumnPtr& get_keys_ptr() { return keys_column; } diff --git a/be/src/vec/columns/column_nullable.cpp b/be/src/vec/columns/column_nullable.cpp index c58c78f5611d02..4f8de7f9cfe173 100644 --- a/be/src/vec/columns/column_nullable.cpp +++ b/be/src/vec/columns/column_nullable.cpp @@ -88,6 +88,23 @@ void ColumnNullable::update_crc_with_value(size_t start, size_t end, uint32_t& h } } +void ColumnNullable::update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const { + if (!has_null()) { + nested_column->update_murmur_with_value(start, end, hash, nullptr); + } else { + const auto* __restrict real_null_data = + assert_cast(*null_map).get_data().data(); + hash = HashUtil::SPARK_MURMUR_32_SEED; + for (int i = start; i < end; ++i) { + if (real_null_data[i] != 0) { + hash = HashUtil::murmur_hash3_32_null(hash); + } + } + nested_column->update_murmur_with_value(start, end, hash, real_null_data); + } +} + void ColumnNullable::update_hash_with_value(size_t n, SipHash& hash) const { if (is_null_at(n)) { hash.update(0); @@ -116,6 +133,27 @@ void ColumnNullable::update_crcs_with_value(uint32_t* __restrict hashes, doris:: } } +void ColumnNullable::update_murmurs_with_value(int32_t* __restrict hashes, + doris::PrimitiveType type, int32_t rows, + uint32_t offset, + const uint8_t* __restrict null_data) const { + DCHECK(null_data == nullptr); + auto s = rows; + DCHECK(s == size()); + const auto* __restrict real_null_data = + assert_cast(*null_map).get_data().data(); + if (!has_null()) { + nested_column->update_murmurs_with_value(hashes, type, rows, offset, nullptr); + } else { + for (int i = 0; i < s; ++i) { + if (real_null_data[i] != 0) { + hashes[i] = HashUtil::murmur_hash3_32_null(HashUtil::SPARK_MURMUR_32_SEED); + } + } + nested_column->update_murmurs_with_value(hashes, type, rows, offset, real_null_data); + } +} + void ColumnNullable::update_hashes_with_value(uint64_t* __restrict hashes, const uint8_t* __restrict null_data) const { DCHECK(null_data == nullptr); diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index 84b3ce0f82aadb..5d32c07ef1eea1 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -296,6 +296,8 @@ class ColumnNullable final : public COWHelper, public N const uint8_t* __restrict null_data) const override; void update_crc_with_value(size_t start, size_t end, uint32_t& hash, const uint8_t* __restrict null_data) const override; + void update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const override; void update_hash_with_value(size_t n, SipHash& hash) const override; void update_crcs_with_value(uint32_t* __restrict hash, PrimitiveType type, uint32_t rows, @@ -303,6 +305,9 @@ class ColumnNullable final : public COWHelper, public N const uint8_t* __restrict null_data) const override; void update_hashes_with_value(uint64_t* __restrict hashes, const uint8_t* __restrict null_data) const override; + void update_murmurs_with_value(int32_t* __restrict hash, PrimitiveType type, int32_t rows, + uint32_t offset, + const uint8_t* __restrict null_data) const override; ColumnPtr convert_column_if_overflow() override { nested_column = nested_column->convert_column_if_overflow(); diff --git a/be/src/vec/columns/column_string.cpp b/be/src/vec/columns/column_string.cpp index db0088e67c27b6..e012fa51a0554b 100644 --- a/be/src/vec/columns/column_string.cpp +++ b/be/src/vec/columns/column_string.cpp @@ -286,6 +286,29 @@ void ColumnStr::update_crcs_with_value(uint32_t* __restrict hashes, doris::Pr } } +void ColumnString::update_murmurs_with_value(int32_t* __restrict hashes, doris::PrimitiveType type, + int32_t rows, uint32_t offset, + const uint8_t* __restrict null_data) const { + auto s = rows; + DCHECK(s == size()); + + if (null_data == nullptr) { + for (size_t i = 0; i < s; i++) { + auto data_ref = get_data_at(i); + hashes[i] = HashUtil::murmur_hash3_32(data_ref.data, data_ref.size, + HashUtil::SPARK_MURMUR_32_SEED); + } + } else { + for (size_t i = 0; i < s; i++) { + if (null_data[i] == 0) { + auto data_ref = get_data_at(i); + hashes[i] = HashUtil::murmur_hash3_32(data_ref.data, data_ref.size, + HashUtil::SPARK_MURMUR_32_SEED); + } + } + } +} + template ColumnPtr ColumnStr::filter(const IColumn::Filter& filt, ssize_t result_size_hint) const { if (offsets.size() == 0) { diff --git a/be/src/vec/columns/column_string.h b/be/src/vec/columns/column_string.h index 1674fd90933dbe..044fcec952ce47 100644 --- a/be/src/vec/columns/column_string.h +++ b/be/src/vec/columns/column_string.h @@ -427,6 +427,25 @@ class ColumnStr final : public COWHelper> { } } + void update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const override { + if (null_data) { + for (size_t i = start; i < end; ++i) { + if (null_data[i] == 0) { + auto data_ref = get_data_at(i); + hash = HashUtil::murmur_hash3_32(data_ref.data, data_ref.size, + HashUtil::SPARK_MURMUR_32_SEED); + } + } + } else { + for (size_t i = start; i < end; ++i) { + auto data_ref = get_data_at(i); + hash = HashUtil::murmur_hash3_32(data_ref.data, data_ref.size, + HashUtil::SPARK_MURMUR_32_SEED); + } + } + } + void update_hash_with_value(size_t n, SipHash& hash) const override { size_t string_size = size_at(n); size_t offset = offset_at(n); @@ -440,6 +459,10 @@ class ColumnStr final : public COWHelper> { uint32_t offset, const uint8_t* __restrict null_data) const override; + void update_murmurs_with_value(int32_t* __restrict hashes, PrimitiveType type, int32_t rows, + uint32_t offset, + const uint8_t* __restrict null_data) const override; + void update_hashes_with_value(uint64_t* __restrict hashes, const uint8_t* __restrict null_data) const override { auto s = size(); diff --git a/be/src/vec/columns/column_struct.cpp b/be/src/vec/columns/column_struct.cpp index 3a238a09c0d1d4..c473f3a84769c0 100644 --- a/be/src/vec/columns/column_struct.cpp +++ b/be/src/vec/columns/column_struct.cpp @@ -220,6 +220,13 @@ void ColumnStruct::update_crc_with_value(size_t start, size_t end, uint32_t& has } } +void ColumnStruct::update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const { + for (const auto& column : columns) { + column->update_murmur_with_value(start, end, hash, nullptr); + } +} + void ColumnStruct::update_hashes_with_value(uint64_t* __restrict hashes, const uint8_t* __restrict null_data) const { for (const auto& column : columns) { @@ -235,6 +242,14 @@ void ColumnStruct::update_crcs_with_value(uint32_t* __restrict hash, PrimitiveTy } } +void ColumnStruct::update_murmurs_with_value(int32_t* __restrict hash, PrimitiveType type, + int32_t rows, uint32_t offset, + const uint8_t* __restrict null_data) const { + for (const auto& column : columns) { + column->update_murmurs_with_value(hash, type, rows, offset, null_data); + } +} + void ColumnStruct::insert_indices_from(const IColumn& src, const uint32_t* indices_begin, const uint32_t* indices_end) { const auto& src_concrete = assert_cast(src); diff --git a/be/src/vec/columns/column_struct.h b/be/src/vec/columns/column_struct.h index e9f8014d9db06e..40ebe1f30fc8c9 100644 --- a/be/src/vec/columns/column_struct.h +++ b/be/src/vec/columns/column_struct.h @@ -123,6 +123,8 @@ class ColumnStruct final : public COWHelper { const uint8_t* __restrict null_data) const override; void update_crc_with_value(size_t start, size_t end, uint32_t& hash, const uint8_t* __restrict null_data) const override; + void update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const override; void update_hashes_with_value(uint64_t* __restrict hashes, const uint8_t* __restrict null_data = nullptr) const override; @@ -131,6 +133,10 @@ class ColumnStruct final : public COWHelper { uint32_t offset = 0, const uint8_t* __restrict null_data = nullptr) const override; + void update_murmurs_with_value(int32_t* __restrict hash, PrimitiveType type, int32_t rows, + uint32_t offset = 0, + const uint8_t* __restrict null_data = nullptr) const override; + void insert_indices_from(const IColumn& src, const uint32_t* indices_begin, const uint32_t* indices_end) override; diff --git a/be/src/vec/columns/column_vector.cpp b/be/src/vec/columns/column_vector.cpp index 7a23156063d4e1..97a72265969a2a 100644 --- a/be/src/vec/columns/column_vector.cpp +++ b/be/src/vec/columns/column_vector.cpp @@ -212,6 +212,41 @@ void ColumnVector::update_crcs_with_value(uint32_t* __restrict hashes, Primit } } +template +void ColumnVector::update_murmurs_with_value(int32_t* __restrict hashes, PrimitiveType type, + int32_t rows, uint32_t offset, + const uint8_t* __restrict null_data) const { + auto s = rows; + DCHECK(s == size()); + + if constexpr (!std::is_same_v) { + DO_MURMUR_HASHES_FUNCTION_COLUMN_IMPL(HashUtil::SPARK_MURMUR_32_SEED) + } else { + if (type == TYPE_DATE || type == TYPE_DATETIME) { + char buf[64]; + auto date_convert_do_crc = [&](size_t i) { + const VecDateTimeValue& date_val = (const VecDateTimeValue&)data[i]; + auto len = date_val.to_buffer(buf); + hashes[i] = HashUtil::murmur_hash3_32(buf, len, HashUtil::SPARK_MURMUR_32_SEED); + }; + + if (null_data == nullptr) { + for (size_t i = 0; i < s; i++) { + date_convert_do_crc(i); + } + } else { + for (size_t i = 0; i < s; i++) { + if (null_data[i] == 0) { + date_convert_do_crc(i); + } + } + } + } else { + DO_MURMUR_HASHES_FUNCTION_COLUMN_IMPL(HashUtil::SPARK_MURMUR_32_SEED) + } + } +} + template struct ColumnVector::less { const Self& parent; diff --git a/be/src/vec/columns/column_vector.h b/be/src/vec/columns/column_vector.h index c14361c6572245..4d038958b74a67 100644 --- a/be/src/vec/columns/column_vector.h +++ b/be/src/vec/columns/column_vector.h @@ -306,12 +306,35 @@ class ColumnVector final : public COWHelper> { } } } + + void update_murmur_with_value(size_t start, size_t end, int32_t& hash, + const uint8_t* __restrict null_data) const override { + if (hash == 0) { + hash = HashUtil::SPARK_MURMUR_32_SEED; + } + if (null_data) { + for (size_t i = start; i < end; i++) { + if (null_data[i] == 0) { + hash = HashUtil::murmur_hash3_32(&data[i], sizeof(T), hash); + } + } + } else { + for (size_t i = start; i < end; i++) { + hash = HashUtil::murmur_hash3_32(&data[i], sizeof(T), hash); + } + } + } + void update_hash_with_value(size_t n, SipHash& hash) const override; void update_crcs_with_value(uint32_t* __restrict hashes, PrimitiveType type, uint32_t rows, uint32_t offset, const uint8_t* __restrict null_data) const override; + void update_murmurs_with_value(int32_t* __restrict hashes, PrimitiveType type, int32_t rows, + uint32_t offset, + const uint8_t* __restrict null_data) const override; + void update_hashes_with_value(uint64_t* __restrict hashes, const uint8_t* __restrict null_data) const override; diff --git a/be/src/vec/runtime/partitioner.cpp b/be/src/vec/runtime/partitioner.cpp index 2654026ed918f1..4a29e9689f722c 100644 --- a/be/src/vec/runtime/partitioner.cpp +++ b/be/src/vec/runtime/partitioner.cpp @@ -26,9 +26,10 @@ namespace doris::vectorized { #include "common/compile_check_begin.h" -template -Status Crc32HashPartitioner::do_partitioning(RuntimeState* state, Block* block) const { - size_t rows = block->rows(); +template +Status Partitioner::do_partitioning(RuntimeState* state, Block* block, + MemTracker* mem_tracker) const { + int rows = block->rows(); if (rows > 0) { auto column_to_keep = block->columns(); @@ -64,6 +65,13 @@ void Crc32HashPartitioner::_do_hash(const ColumnPtr& column, cast_set(column->size())); } +template +void Murmur32HashPartitioner::_do_hash(const ColumnPtr& column, + int32_t* __restrict result, int idx) const { + column->update_murmurs_with_value(result, Base::_partition_expr_ctxs[idx]->root()->type().type, + cast_set(column->size())); +} + template Status Crc32HashPartitioner::clone(RuntimeState* state, std::unique_ptr& partitioner) { @@ -77,7 +85,21 @@ Status Crc32HashPartitioner::clone(RuntimeState* state, return Status::OK(); } +template +Status Murmur32HashPartitioner::clone(RuntimeState* state, + std::unique_ptr& partitioner) { + auto* new_partitioner = new Murmur32HashPartitioner(Base::_partition_count); + partitioner.reset(new_partitioner); + new_partitioner->_partition_expr_ctxs.resize(Base::_partition_expr_ctxs.size()); + for (size_t i = 0; i < Base::_partition_expr_ctxs.size(); i++) { + RETURN_IF_ERROR(Base::_partition_expr_ctxs[i]->clone( + state, new_partitioner->_partition_expr_ctxs[i])); + } + return Status::OK(); +} + template class Crc32HashPartitioner; template class Crc32HashPartitioner; +template class Murmur32HashPartitioner; } // namespace doris::vectorized diff --git a/be/src/vec/runtime/partitioner.h b/be/src/vec/runtime/partitioner.h index 53d8b84d09c752..6556b91d6ef4a6 100644 --- a/be/src/vec/runtime/partitioner.h +++ b/be/src/vec/runtime/partitioner.h @@ -111,5 +111,23 @@ struct SpillPartitionChannelIds { return ((l >> 16) | (l << 16)) % r; } }; + +template +class Murmur32HashPartitioner final : public Partitioner { +public: + using Base = Partitioner; + Murmur32HashPartitioner(int partition_count) + : Partitioner(partition_count) {} + ~Murmur32HashPartitioner() override = default; + + Status clone(RuntimeState* state, std::unique_ptr& partitioner) override; + +private: + void _do_hash(const ColumnPtr& column, int32_t* __restrict result, int idx) const override; +}; + +} // namespace vectorized +} // namespace doris + #include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/sink/vdata_stream_sender.h b/be/src/vec/sink/vdata_stream_sender.h index 0ff1f252d5441f..8c3aab6622b693 100644 --- a/be/src/vec/sink/vdata_stream_sender.h +++ b/be/src/vec/sink/vdata_stream_sender.h @@ -100,6 +100,13 @@ class BlockSerializer { const int _batch_size; }; +struct ShufflePModChannelIds { + template + HashValueType operator()(HashValueType l, int32_t r) { + return (l % r + r) % r; + } +}; + class Channel { public: friend class pipeline::ExchangeSinkBuffer; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveExternalDistributionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveExternalDistributionInfo.java new file mode 100644 index 00000000000000..5b15874401908a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveExternalDistributionInfo.java @@ -0,0 +1,95 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.catalog; + +import com.google.gson.annotations.SerializedName; + +import java.util.List; +import java.util.Objects; + +/* + * Hive Hash Distribution Info + */ +public class HiveExternalDistributionInfo extends HashDistributionInfo { + @SerializedName(value = "bucketingVersion") + private final int bucketingVersion; + + public HiveExternalDistributionInfo() { + bucketingVersion = 2; + } + + public HiveExternalDistributionInfo(int bucketNum, List distributionColumns, int bucketingVersion) { + super(bucketNum, distributionColumns); + this.bucketingVersion = bucketingVersion; + } + + public HiveExternalDistributionInfo(int bucketNum, boolean autoBucket, + List distributionColumns, int bucketingVersion) { + super(bucketNum, autoBucket, distributionColumns); + this.bucketingVersion = bucketingVersion; + } + + public int getBucketingVersion() { + return bucketingVersion; + } + + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + HiveExternalDistributionInfo that = (HiveExternalDistributionInfo) o; + return bucketNum == that.bucketNum + && sameDistributionColumns(that) + && bucketingVersion == that.bucketingVersion; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), bucketingVersion); + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("type: ").append(type).append("; "); + + builder.append("distribution columns: ["); + for (Column column : getDistributionColumns()) { + builder.append(column.getName()).append(","); + } + builder.append("]; "); + + if (autoBucket) { + builder.append("bucket num: auto;"); + } else { + builder.append("bucket num: ").append(bucketNum).append(";"); + } + + builder.append("bucketingVersion: ").append(bucketingVersion).append(";"); + + return builder.toString(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/FederationBackendPolicy.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/FederationBackendPolicy.java index 4a24645bf3ee03..a7f6b9431e0104 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/FederationBackendPolicy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/FederationBackendPolicy.java @@ -203,6 +203,8 @@ public void init(BeSelectionPolicy policy) throws UserException { } catch (ExecutionException e) { throw new UserException("failed to get consistent hash", e); } + /*consistentBucket = new ConsistentHash<>(Hashing.murmur3_128(), new BucketHash(), + new BackendHash(), backends, Config.virtual_node_number);*/ } public Backend getNextBe() { @@ -248,6 +250,7 @@ public Multimap computeScanRangeAssignment(List splits) t Optional chosenNode = candidateNodes.stream() .min(Comparator.comparingLong(ownerNode -> assignedWeightPerBackend.get(ownerNode))); + //ToDo(Nitin): group assignment based on the bucketId if (chosenNode.isPresent()) { Backend selectedBackend = chosenNode.get(); assignment.put(selectedBackend, split); @@ -505,4 +508,11 @@ public void funnel(Split split, PrimitiveSink primitiveSink) { primitiveSink.putLong(split.getLength()); } } + + private static class BucketHash implements Funnel { + @Override + public void funnel(Integer bucketId, PrimitiveSink primitiveSink) { + primitiveSink.putLong(bucketId); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java index 69115e969b0ba3..c41c6e6aabd342 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java @@ -35,8 +35,11 @@ import org.apache.doris.common.util.Util; import org.apache.doris.datasource.hive.AcidInfo; import org.apache.doris.datasource.hive.AcidInfo.DeleteDeltaInfo; +import org.apache.doris.datasource.hive.HMSExternalTable; +import org.apache.doris.datasource.hive.HiveBucketUtil; import org.apache.doris.datasource.hive.source.HiveScanNode; import org.apache.doris.datasource.hive.source.HiveSplit; +import org.apache.doris.planner.DataPartition; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; @@ -53,6 +56,7 @@ import org.apache.doris.thrift.TFileScanRangeParams; import org.apache.doris.thrift.TFileScanSlotInfo; import org.apache.doris.thrift.TFileType; +import org.apache.doris.thrift.THashType; import org.apache.doris.thrift.THdfsParams; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TScanRange; @@ -65,6 +69,7 @@ import org.apache.doris.thrift.TTransactionalHiveDesc; import com.google.common.base.Preconditions; +import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; @@ -89,6 +94,8 @@ public abstract class FileQueryScanNode extends FileScanNode { protected Map destSlotDescByName; protected TFileScanRangeParams params; + public ArrayListMultimap bucketSeq2locations = ArrayListMultimap.create(); + @Getter protected TableSample tableSample; @@ -353,13 +360,76 @@ public void createScanRangeLocations() throws UserException { tSource.setSplitSourceId(splitSource.getUniqueId()); tSource.setNumSplits(numSplitsPerBE); curLocations.getScanRange().getExtScanRange().getFileScanRange().setSplitSource(tSource); +/*======= + // If fileSplit has partition values, use the values collected from hive partitions. + // Otherwise, use the values in file path. + boolean isACID = false; + if (fileSplit instanceof HiveSplit) { + HiveSplit hiveSplit = (HiveSplit) fileSplit; + isACID = hiveSplit.isACID(); + } + List partitionValuesFromPath = fileSplit.getPartitionValues() == null + ? BrokerUtil.parseColumnsFromPath(fileSplit.getPath().toString(), pathPartitionKeys, + false, isACID) : fileSplit.getPartitionValues(); + + boolean isBucketedHiveTable = false; + int bucketNum = 0; + TableIf targetTable = getTargetTable(); + if (targetTable instanceof HMSExternalTable) { + isBucketedHiveTable = ((HMSExternalTable) targetTable).isBucketedTable(); + if (isBucketedHiveTable) { + bucketNum = HiveBucketUtil.getBucketNumberFromPath(fileSplit.getPath().getName()).getAsInt(); + } + } + TFileRangeDesc rangeDesc = createFileRangeDesc(fileSplit, partitionValuesFromPath, pathPartitionKeys, + locationType); + TFileCompressType fileCompressType = getFileCompressType(fileSplit); + rangeDesc.setCompressType(fileCompressType); + if (isACID) { + HiveSplit hiveSplit = (HiveSplit) fileSplit; + hiveSplit.setTableFormatType(TableFormatType.TRANSACTIONAL_HIVE); + TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); + tableFormatFileDesc.setTableFormatType(hiveSplit.getTableFormatType().value()); + AcidInfo acidInfo = (AcidInfo) hiveSplit.getInfo(); + TTransactionalHiveDesc transactionalHiveDesc = new TTransactionalHiveDesc(); + transactionalHiveDesc.setPartition(acidInfo.getPartitionLocation()); + List deleteDeltaDescs = new ArrayList<>(); + for (DeleteDeltaInfo deleteDeltaInfo : acidInfo.getDeleteDeltas()) { + TTransactionalHiveDeleteDeltaDesc deleteDeltaDesc = new TTransactionalHiveDeleteDeltaDesc(); + deleteDeltaDesc.setDirectoryLocation(deleteDeltaInfo.getDirectoryLocation()); + deleteDeltaDesc.setFileNames(deleteDeltaInfo.getFileNames()); + deleteDeltaDescs.add(deleteDeltaDesc); + } + transactionalHiveDesc.setDeleteDeltas(deleteDeltaDescs); + tableFormatFileDesc.setTransactionalHiveParams(transactionalHiveDesc); + rangeDesc.setTableFormatParams(tableFormatFileDesc); + } + + setScanParams(rangeDesc, fileSplit); + + curLocations.getScanRange().getExtScanRange().getFileScanRange().addToRanges(rangeDesc); +>>>>>>> a5ce2395a2 ([feature](datalake) Add BucketShuffleJoin support for Hive table data generated by Spark. (27783)) +*/ TScanRangeLocation location = new TScanRangeLocation(); location.setBackendId(backend.getId()); location.setServer(new TNetworkAddress(backend.getHost(), backend.getBePort())); curLocations.addToLocations(location); - // So there's only one scan range for each backend. + + // So there's only one scan range for each backend. // Each backend only starts up one ScanNode instance. // However, even one ScanNode instance can provide maximum scanning concurrency. +/*======= + if (LOG.isDebugEnabled()) { + LOG.debug("assign to backend {} with table split: {} ({}, {}), location: {}", + curLocations.getLocations().get(0).getBackendId(), fileSplit.getPath(), + fileSplit.getStart(), fileSplit.getLength(), + Joiner.on("|").join(fileSplit.getHosts())); + } + if (isBucketedHiveTable) { + bucketSeq2locations.put(bucketNum, curLocations); + } +>>>>>>> a5ce2395a2 ([feature](datalake) Add BucketShuffleJoin support for Hive table data generated by Spark. (27783)) +*/ scanRangeLocations.add(curLocations); setLocationPropertiesIfNecessary(backend, locationType, locationProperties); scanBackendIds.add(backend.getId()); @@ -592,6 +662,14 @@ protected TFileAttributes getFileAttributes() throws UserException { protected abstract Map getLocationProperties() throws UserException; + public DataPartition constructInputPartitionByDistributionInfo() { + return DataPartition.RANDOM; + } + + public THashType getHashType() { + return THashType.CRC32; + } + @Override public void stop() { if (splitAssignment != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index 71c7308b079866..c3757ef5a6593e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -19,12 +19,16 @@ import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.DistributionInfo; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.HashDistributionInfo; +import org.apache.doris.catalog.HiveExternalDistributionInfo; import org.apache.doris.catalog.ListPartitionItem; import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.PartitionItem; import org.apache.doris.catalog.PartitionType; import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.RandomDistributionInfo; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; @@ -73,6 +77,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.LongColumnStatsData; import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.StringColumnStatsData; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -80,8 +85,10 @@ import org.apache.logging.log4j.Logger; import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.time.LocalDate; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -108,6 +115,22 @@ public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableI private static final String TBL_PROP_TRANSIENT_LAST_DDL_TIME = "transient_lastDdlTime"; private static final String NUM_ROWS = "numRows"; + private static final String SPARK_BUCKET = "spark.sql.sources.schema.bucketCol."; + private static final String SPARK_NUM_BUCKET = "spark.sql.sources.schema.numBuckets"; + private static final String BUCKETING_VERSION = "bucketing_version"; + + private static final Set SUPPORTED_BUCKET_PROPERTIES; + + static { + SUPPORTED_BUCKET_PROPERTIES = Sets.newHashSet(); + SUPPORTED_BUCKET_PROPERTIES.add(SPARK_BUCKET + "0"); + SUPPORTED_BUCKET_PROPERTIES.add(SPARK_BUCKET + "1"); + SUPPORTED_BUCKET_PROPERTIES.add(SPARK_BUCKET + "2"); + SUPPORTED_BUCKET_PROPERTIES.add(SPARK_BUCKET + "3"); + SUPPORTED_BUCKET_PROPERTIES.add(SPARK_BUCKET + "4"); + SUPPORTED_BUCKET_PROPERTIES.add(SPARK_NUM_BUCKET); + SUPPORTED_BUCKET_PROPERTIES.add(BUCKETING_VERSION); + } private static final String SPARK_COL_STATS = "spark.sql.statistics.colStats."; private static final String SPARK_STATS_MAX = ".max"; @@ -154,7 +177,10 @@ public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableI MAP_SPARK_STATS_TO_DORIS.put(StatsType.HISTOGRAM, SPARK_STATS_HISTOGRAM); } - private volatile org.apache.hadoop.hive.metastore.api.Table remoteTable = null; + protected volatile org.apache.hadoop.hive.metastore.api.Table remoteTable = null; + protected List partitionColumns; + private List bucketColumns; + private boolean isSparkTable; private DLAType dlaType = DLAType.UNKNOWN; @@ -165,6 +191,8 @@ public enum DLAType { UNKNOWN, HIVE, HUDI, ICEBERG } + private DistributionInfo distributionInfo; + /** * Create hive metastore external table. * @@ -244,6 +272,14 @@ public boolean isHoodieCowTable() { || (params != null && "COPY_ON_WRITE".equalsIgnoreCase(params.get("flink.table.type"))); } + public boolean isSparkTable() { + return isSparkTable; + } + + public boolean isBucketedTable() { + return bucketColumns != null && !bucketColumns.isEmpty() && isSparkTable; + } + /** * Some data lakes (such as Hudi) will synchronize their partition information to HMS, * then we can quickly obtain the partition information of the table from HMS. @@ -546,9 +582,71 @@ public Optional initSchema() { private Optional getIcebergSchema() { List columns = IcebergUtils.getSchema(catalog, dbName, name); List partitionColumns = initPartitionColumns(columns); + initBucketingColumns(columns); return Optional.of(new HMSSchemaCacheValue(columns, partitionColumns)); } + private void initBucketingColumns(List columns) { + List bucketCols = new ArrayList<>(5); + int numBuckets = getBucketColumns(bucketCols); + if (bucketCols.isEmpty() || !isSparkTable) { + bucketColumns = ImmutableList.of(); + distributionInfo = new RandomDistributionInfo(1, true); + return; + } + + int bucketingVersion = Integer.valueOf(remoteTable.getParameters().getOrDefault(BUCKETING_VERSION, "2")); + ImmutableList.Builder bucketColBuilder = ImmutableList.builder(); + for (String colName : bucketCols) { + // do not use "getColumn()", which will cause dead loop + for (Column column : columns) { + if (colName.equalsIgnoreCase(column.getName())) { + // For partition/bucket column, if it is string type, change it to varchar(65535) + // to be same as doris managed table. + // This is to avoid some unexpected behavior such as different partition pruning result + // between doris managed table and external table. + if (column.getType().getPrimitiveType() == PrimitiveType.STRING) { + column.setType(ScalarType.createVarcharType(ScalarType.MAX_VARCHAR_LENGTH)); + } + bucketColBuilder.add(column); + break; + } + } + } + + bucketColumns = bucketColBuilder.build(); + distributionInfo = new HiveExternalDistributionInfo(numBuckets, bucketColumns, bucketingVersion); + LOG.debug("get {} bucket columns for table: {}", bucketColumns.size(), name); + } + + private int getBucketColumns(List bucketCols) { + StorageDescriptor descriptor = remoteTable.getSd(); + int numBuckets = -1; + if (descriptor.isSetBucketCols() && !descriptor.getBucketCols().isEmpty()) { + /* Hive Bucketed Table */ + bucketCols.addAll(descriptor.getBucketCols()); + numBuckets = descriptor.getNumBuckets(); + } else if (remoteTable.isSetParameters() + && !Collections.disjoint(SUPPORTED_BUCKET_PROPERTIES, remoteTable.getParameters().keySet())) { + Map parameters = remoteTable.getParameters(); + for (Map.Entry param : parameters.entrySet()) { + if (param.getKey().startsWith(SPARK_BUCKET)) { + int index = Integer.valueOf(param.getKey() + .substring(param.getKey().lastIndexOf(".") + 1)); + bucketCols.add(index, param.getValue()); + } else if (param.getKey().equals(SPARK_NUM_BUCKET)) { + numBuckets = Integer.valueOf(param.getValue()); + } + } + + if (numBuckets > 0) { + isSparkTable = true; + } + } + + return numBuckets; + } + private Optional getHudiSchema() { org.apache.avro.Schema hudiSchema = HiveMetaStoreClientHelper.getHudiTableSchema(this); List tmpSchema = Lists.newArrayListWithCapacity(hudiSchema.getFields().size()); @@ -661,6 +759,19 @@ public Optional getColumnStatistic(String colName) { return Optional.empty(); } + public DistributionInfo getDefaultDistributionInfo() { + makeSureInitialized(); + if (distributionInfo != null) { + return distributionInfo; + } + + return new RandomDistributionInfo(1, true); + } + + public Map getTableParameters() { + return remoteTable.getParameters(); + } + private Optional getHiveColumnStats(String colName) { List tableStats = getHiveTableColumnStats(Lists.newArrayList(colName)); if (tableStats == null || tableStats.isEmpty()) { @@ -782,14 +893,23 @@ public long getDataSize(boolean singleReplica) { @Override public boolean isDistributionColumn(String columnName) { - return getRemoteTable().getSd().getBucketCols().stream().map(String::toLowerCase) - .collect(Collectors.toSet()).contains(columnName.toLowerCase()); + Set distributeColumns = getDistributionColumnNames() + .stream().map(String::toLowerCase).collect(Collectors.toSet()); + return distributeColumns.contains(columnName.toLowerCase()); } @Override public Set getDistributionColumnNames() { - return getRemoteTable().getSd().getBucketCols().stream().map(String::toLowerCase) - .collect(Collectors.toSet()); + Set distributionColumnNames = Sets.newHashSet(); + if (distributionInfo instanceof RandomDistributionInfo) { + return distributionColumnNames; + } + HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo; + List distColumn = hashDistributionInfo.getDistributionColumns(); + for (Column column : distColumn) { + distributionColumnNames.add(column.getName().toLowerCase()); + } + return distributionColumnNames; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveBucketUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveBucketUtil.java index 7435a3d58dc911..ce0d9cfba98bf7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveBucketUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveBucketUtil.java @@ -96,6 +96,9 @@ private static PrimitiveTypeInfo convertToHiveColType(PrimitiveType dorisType) t Pattern.compile("bucket_(\\d+)(_\\d+)?$"); private static final Iterable BUCKET_PATTERNS = ImmutableList.of( + // spark/parquet pattern + // format: f"part-[paritionId]-[tid]-[txnId]-[jobId]-[taskAttemptId]-[fileCount].c000.snappy.parquet" + Pattern.compile("part-\\d{5}-\\w{8}-\\w{4}-\\w{4}-\\w{4}-\\w{12}_(\\d{5})(?:[-_.].*)?"), // legacy Presto naming pattern (current version matches Hive) Pattern.compile("\\d{8}_\\d{6}_\\d{5}_[a-z0-9]{5}_bucket-(\\d+)(?:[-_.].*)?"), // Hive naming pattern per `org.apache.hadoop.hive.ql.exec.Utilities#getBucketIdFromFile()` @@ -398,7 +401,7 @@ private static int hashCodeV2(Object o, ObjectInspector objIns, ByteBuffer byteB throw new DdlException("Unknown type: " + objIns.getTypeName()); } - private static OptionalInt getBucketNumberFromPath(String name) { + public static OptionalInt getBucketNumberFromPath(String name) { for (Pattern pattern : BUCKET_PATTERNS) { Matcher matcher = pattern.matcher(name); if (matcher.matches()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java index 3a8ab722fb68bf..527c141da8d10e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java @@ -17,10 +17,15 @@ package org.apache.doris.datasource.hive.source; +import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.FunctionCallExpr; +import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.DistributionInfo; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.HashDistributionInfo; +import org.apache.doris.catalog.HiveExternalDistributionInfo; import org.apache.doris.catalog.ListPartitionItem; import org.apache.doris.catalog.PartitionItem; import org.apache.doris.catalog.TableIf; @@ -43,6 +48,7 @@ import org.apache.doris.datasource.hive.HiveTransaction; import org.apache.doris.datasource.hive.source.HiveSplit.HiveSplitCreator; import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; +import org.apache.doris.planner.DataPartition; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; @@ -52,6 +58,7 @@ import org.apache.doris.thrift.TFileCompressType; import org.apache.doris.thrift.TFileFormatType; import org.apache.doris.thrift.TFileTextScanRangeParams; +import org.apache.doris.thrift.THashType; import org.apache.doris.thrift.TPushAggOp; import com.google.common.base.Preconditions; @@ -497,5 +504,38 @@ protected TFileCompressType getFileCompressType(FileSplit fileSplit) throws User } return compressType; } + + @Override + public DataPartition constructInputPartitionByDistributionInfo() { + if (hmsTable.isBucketedTable()) { + DistributionInfo distributionInfo = hmsTable.getDefaultDistributionInfo(); + if (!(distributionInfo instanceof HashDistributionInfo)) { + return DataPartition.RANDOM; + } + List distributeColumns = ((HiveExternalDistributionInfo) distributionInfo).getDistributionColumns(); + List dataDistributeExprs = Lists.newArrayList(); + for (Column column : distributeColumns) { + SlotRef slotRef = new SlotRef(desc.getRef().getName(), column.getName()); + dataDistributeExprs.add(slotRef); + } + return DataPartition.hashPartitioned(dataDistributeExprs, THashType.SPARK_MURMUR32); + } + + return DataPartition.RANDOM; + } + + public HMSExternalTable getHiveTable() { + return hmsTable; + } + + @Override + public THashType getHashType() { + if (hmsTable.isBucketedTable() + && hmsTable.getDefaultDistributionInfo() instanceof HashDistributionInfo) { + return THashType.SPARK_MURMUR32; + } + + return THashType.CRC32; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 85243c4b545420..202a203544fd9d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -200,6 +200,7 @@ import org.apache.doris.statistics.StatisticConstants; import org.apache.doris.tablefunction.TableValuedFunctionIf; import org.apache.doris.thrift.TFetchOption; +import org.apache.doris.thrift.THashType; import org.apache.doris.thrift.TPartitionType; import org.apache.doris.thrift.TPushAggOp; import org.apache.doris.thrift.TResultSinkType; @@ -556,7 +557,8 @@ public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTransla SessionVariable sv = ConnectContext.get().getSessionVariable(); // TODO(cmy): determine the needCheckColumnPriv param - ScanNode scanNode; + FileQueryScanNode scanNode; + DataPartition dataPartition = DataPartition.RANDOM; if (table instanceof HMSExternalTable) { switch (((HMSExternalTable) table).getDlaType()) { case ICEBERG: @@ -683,8 +685,16 @@ private PlanFragment getPlanFragmentForPhysicalFileScan(PhysicalFileScan fileSca ) ); context.getTopnFilterContext().translateTarget(fileScan, scanNode, context); - // Create PlanFragment + Utils.execWithUncheckedException(scanNode::finalizeForNereids); DataPartition dataPartition = DataPartition.RANDOM; + if (fileScan.getDistributionSpec() instanceof DistributionSpecHash) { + DistributionSpecHash distributionSpecHash = (DistributionSpecHash) fileScan.getDistributionSpec(); + List partitionExprs = distributionSpecHash.getOrderedShuffledColumns().stream() + .map(context::findSlotRef).collect(Collectors.toList()); + dataPartition = new DataPartition(TPartitionType.HASH_PARTITIONED, + partitionExprs, scanNode.getHashType()); + } + // Create PlanFragment PlanFragment planFragment = createPlanFragment(scanNode, dataPartition, fileScan); context.addPlanFragment(planFragment); updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), fileScan); @@ -2604,7 +2614,7 @@ private void addPlanRoot(PlanFragment fragment, PlanNode planNode, AbstractPlan } private DataPartition toDataPartition(DistributionSpec distributionSpec, - List childOutputIds, PlanTranslatorContext context) { + List childOutputIds, PlanTranslatorContext context) { if (distributionSpec instanceof DistributionSpecAny || distributionSpec instanceof DistributionSpecStorageAny || distributionSpec instanceof DistributionSpecExecutionAny) { @@ -2631,8 +2641,20 @@ private DataPartition toDataPartition(DistributionSpec distributionSpec, } } TPartitionType partitionType; + THashType hashType = THashType.XXHASH64; switch (distributionSpecHash.getShuffleType()) { case STORAGE_BUCKETED: + switch (distributionSpecHash.getShuffleFunction()) { + case STORAGE_BUCKET_SPARK_MURMUR32: + hashType = THashType.SPARK_MURMUR32; + break; + case STORAGE_BUCKET_CRC32: + hashType = THashType.CRC32; + break; + case STORAGE_BUCKET_XXHASH64: + default: + break; + } partitionType = TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED; break; case EXECUTION_BUCKETED: @@ -2643,7 +2665,7 @@ private DataPartition toDataPartition(DistributionSpec distributionSpec, throw new RuntimeException("Do not support shuffle type: " + distributionSpecHash.getShuffleType()); } - return new DataPartition(partitionType, partitionExprs); + return new DataPartition(partitionType, partitionExprs, hashType); } else if (distributionSpec instanceof DistributionSpecTabletIdShuffle) { return DataPartition.TABLET_ID; } else if (distributionSpec instanceof DistributionSpecTableSinkHashPartitioned) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java index 8f191b61286e43..7b9d696cabc5e9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java @@ -143,7 +143,7 @@ public PhysicalProperties visitPhysicalEsScan(PhysicalEsScan esScan, PlanContext @Override public PhysicalProperties visitPhysicalFileScan(PhysicalFileScan fileScan, PlanContext context) { - return PhysicalProperties.STORAGE_ANY; + return new PhysicalProperties(fileScan.getDistributionSpec()); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java index b821ff0de87ae0..d786215692cbb2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java @@ -679,7 +679,7 @@ private PhysicalProperties calAnotherSideRequired(ShuffleType shuffleType, notNeedShuffleSideRequired, needShuffleSideRequired); return new PhysicalProperties(new DistributionSpecHash(shuffleSideIds, shuffleType, needShuffleSideOutput.getTableId(), needShuffleSideOutput.getSelectedIndexId(), - needShuffleSideOutput.getPartitionIds())); + needShuffleSideOutput.getPartitionIds(), notShuffleSideOutput.getShuffleFunction())); } private void updateChildEnforceAndCost(int index, PhysicalProperties targetProperties) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java index 6ab8e054f8aaef..5bf1a7f52472bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java @@ -54,6 +54,8 @@ public class DistributionSpecHash extends DistributionSpec { private final Set partitionIds; private final long selectedIndexId; + private final StorageBucketHashType storageBucketHashType; + /** * Use for no need set table related attributes. */ @@ -70,10 +72,19 @@ public DistributionSpecHash(List orderedShuffledColumns, ShuffleType shu } /** - * Normal constructor. + * Use when no need set shuffle hash function */ public DistributionSpecHash(List orderedShuffledColumns, ShuffleType shuffleType, long tableId, long selectedIndexId, Set partitionIds) { + this(orderedShuffledColumns, shuffleType, tableId, selectedIndexId, partitionIds, + StorageBucketHashType.STORAGE_BUCKET_CRC32); + } + + /** + * Normal constructor. + */ + public DistributionSpecHash(List orderedShuffledColumns, ShuffleType shuffleType, + long tableId, long selectedIndexId, Set partitionIds, StorageBucketHashType storageBucketHashType) { this.orderedShuffledColumns = ImmutableList.copyOf( Objects.requireNonNull(orderedShuffledColumns, "orderedShuffledColumns should not null")); this.shuffleType = Objects.requireNonNull(shuffleType, "shuffleType should not null"); @@ -92,6 +103,7 @@ public DistributionSpecHash(List orderedShuffledColumns, ShuffleType shu } this.equivalenceExprIds = equivalenceExprIdsBuilder.build(); this.exprIdToEquivalenceSet = exprIdToEquivalenceSetBuilder.buildKeepingLast(); + this.storageBucketHashType = storageBucketHashType; } /** @@ -101,7 +113,7 @@ public DistributionSpecHash(List orderedShuffledColumns, ShuffleType shu long tableId, Set partitionIds, List> equivalenceExprIds, Map exprIdToEquivalenceSet) { this(orderedShuffledColumns, shuffleType, tableId, -1L, partitionIds, - equivalenceExprIds, exprIdToEquivalenceSet); + equivalenceExprIds, exprIdToEquivalenceSet, StorageBucketHashType.STORAGE_BUCKET_XXHASH64); } /** @@ -109,7 +121,7 @@ public DistributionSpecHash(List orderedShuffledColumns, ShuffleType shu */ public DistributionSpecHash(List orderedShuffledColumns, ShuffleType shuffleType, long tableId, long selectedIndexId, Set partitionIds, List> equivalenceExprIds, - Map exprIdToEquivalenceSet) { + Map exprIdToEquivalenceSet, StorageBucketHashType storageBucketHashType) { this.orderedShuffledColumns = ImmutableList.copyOf(Objects.requireNonNull(orderedShuffledColumns, "orderedShuffledColumns should not null")); this.shuffleType = Objects.requireNonNull(shuffleType, "shuffleType should not null"); @@ -121,6 +133,7 @@ public DistributionSpecHash(List orderedShuffledColumns, ShuffleType shu Objects.requireNonNull(equivalenceExprIds, "equivalenceExprIds should not null")); this.exprIdToEquivalenceSet = ImmutableMap.copyOf( Objects.requireNonNull(exprIdToEquivalenceSet, "exprIdToEquivalenceSet should not null")); + this.storageBucketHashType = storageBucketHashType; } static DistributionSpecHash merge(DistributionSpecHash left, DistributionSpecHash right, ShuffleType shuffleType) { @@ -140,7 +153,7 @@ static DistributionSpecHash merge(DistributionSpecHash left, DistributionSpecHas exprIdToEquivalenceSet.putAll(right.getExprIdToEquivalenceSet()); return new DistributionSpecHash(orderedShuffledColumns, shuffleType, left.getTableId(), left.getSelectedIndexId(), left.getPartitionIds(), equivalenceExprIds.build(), - exprIdToEquivalenceSet.buildKeepingLast()); + exprIdToEquivalenceSet.buildKeepingLast(), left.getShuffleFunction()); } static DistributionSpecHash merge(DistributionSpecHash left, DistributionSpecHash right) { @@ -175,6 +188,10 @@ public Map getExprIdToEquivalenceSet() { return exprIdToEquivalenceSet; } + public StorageBucketHashType getShuffleFunction() { + return storageBucketHashType; + } + public Set getEquivalenceExprIdsOf(ExprId exprId) { if (exprIdToEquivalenceSet.containsKey(exprId)) { return equivalenceExprIds.get(exprIdToEquivalenceSet.get(exprId)); @@ -227,14 +244,15 @@ private boolean equalsSatisfy(List required) { return true; } - public DistributionSpecHash withShuffleType(ShuffleType shuffleType) { + public DistributionSpecHash withShuffleType(ShuffleType shuffleType, StorageBucketHashType storageBucketHashType) { return new DistributionSpecHash(orderedShuffledColumns, shuffleType, tableId, selectedIndexId, partitionIds, - equivalenceExprIds, exprIdToEquivalenceSet); + equivalenceExprIds, exprIdToEquivalenceSet, storageBucketHashType); } - public DistributionSpecHash withShuffleTypeAndForbidColocateJoin(ShuffleType shuffleType) { + public DistributionSpecHash withShuffleTypeAndForbidColocateJoin(ShuffleType shuffleType, + StorageBucketHashType storageBucketHashType) { return new DistributionSpecHash(orderedShuffledColumns, shuffleType, -1, -1, partitionIds, - equivalenceExprIds, exprIdToEquivalenceSet); + equivalenceExprIds, exprIdToEquivalenceSet, storageBucketHashType); } /** @@ -272,7 +290,7 @@ public DistributionSpec project(Map projections, } } return new DistributionSpecHash(orderedShuffledColumns, shuffleType, tableId, selectedIndexId, partitionIds, - equivalenceExprIds, exprIdToEquivalenceSet); + equivalenceExprIds, exprIdToEquivalenceSet, storageBucketHashType); } @Override @@ -281,12 +299,13 @@ public boolean equals(Object o) { return false; } DistributionSpecHash that = (DistributionSpecHash) o; - return shuffleType == that.shuffleType && orderedShuffledColumns.equals(that.orderedShuffledColumns); + return shuffleType == that.shuffleType && storageBucketHashType == that.storageBucketHashType + && orderedShuffledColumns.equals(that.orderedShuffledColumns); } @Override public int hashCode() { - return Objects.hash(shuffleType, orderedShuffledColumns); + return Objects.hash(shuffleType, storageBucketHashType, orderedShuffledColumns); } @Override @@ -315,4 +334,13 @@ public enum ShuffleType { STORAGE_BUCKETED, } + /** + * Enums for concrete shuffle functions. + */ + public enum StorageBucketHashType { + STORAGE_BUCKET_CRC32, + STORAGE_BUCKET_XXHASH64, + STORAGE_BUCKET_SPARK_MURMUR32 + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/EnforceMissingPropertiesHelper.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/EnforceMissingPropertiesHelper.java index b08db2aeba2b15..c7c40fe1e98189 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/EnforceMissingPropertiesHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/EnforceMissingPropertiesHelper.java @@ -26,6 +26,7 @@ import org.apache.doris.nereids.metrics.event.EnforcerEvent; import org.apache.doris.nereids.minidump.NereidsTracer; import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; +import org.apache.doris.nereids.properties.DistributionSpecHash.StorageBucketHashType; import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; import org.apache.doris.qe.ConnectContext; @@ -117,7 +118,8 @@ private PhysicalProperties enforceDistribution(PhysicalProperties oldOutputPrope DistributionSpec requiredDistributionSpec = required.getDistributionSpec(); if (requiredDistributionSpec instanceof DistributionSpecHash) { DistributionSpecHash requiredDistributionSpecHash = (DistributionSpecHash) requiredDistributionSpec; - outputDistributionSpec = requiredDistributionSpecHash.withShuffleType(ShuffleType.EXECUTION_BUCKETED); + outputDistributionSpec = requiredDistributionSpecHash.withShuffleType(ShuffleType.EXECUTION_BUCKETED, + StorageBucketHashType.STORAGE_BUCKET_XXHASH64); } else { outputDistributionSpec = requiredDistributionSpec; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java index 70ab9b1d502c0d..a0fa806ed2427b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java @@ -17,12 +17,28 @@ package org.apache.doris.nereids.rules.implementation; -import org.apache.doris.nereids.properties.DistributionSpecAny; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.DistributionInfo; +import org.apache.doris.catalog.HashDistributionInfo; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.datasource.hive.HMSExternalTable; +import org.apache.doris.nereids.properties.DistributionSpec; +import org.apache.doris.nereids.properties.DistributionSpecHash; +import org.apache.doris.nereids.properties.DistributionSpecHash.StorageBucketHashType; +import org.apache.doris.nereids.properties.DistributionSpecStorageAny; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.plans.logical.LogicalHudiScan; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalFileScan; +import com.google.common.collect.Lists; + +import java.util.Collections; +import java.util.List; import java.util.Optional; /** @@ -36,7 +52,7 @@ public Rule build() { fileScan.getRelationId(), fileScan.getTable(), fileScan.getQualifier(), - DistributionSpecAny.INSTANCE, + convertDistribution(fileScan), Optional.empty(), fileScan.getLogicalProperties(), fileScan.getSelectedPartitions(), @@ -44,4 +60,34 @@ public Rule build() { fileScan.getTableSnapshot()) ).toRule(RuleType.LOGICAL_FILE_SCAN_TO_PHYSICAL_FILE_SCAN_RULE); } + + private DistributionSpec convertDistribution(LogicalFileScan fileScan) { + TableIf table = fileScan.getTable(); + if (!(table instanceof HMSExternalTable)) { + return DistributionSpecStorageAny.INSTANCE; + } + + HMSExternalTable hmsExternalTable = (HMSExternalTable) table; + DistributionInfo distributionInfo = hmsExternalTable.getDefaultDistributionInfo(); + if (distributionInfo instanceof HashDistributionInfo) { + HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo; + List output = fileScan.getOutput(); + List hashColumns = Lists.newArrayList(); + for (Slot slot : output) { + for (Column column : hashDistributionInfo.getDistributionColumns()) { + if (((SlotReference) slot).getColumn().get().equals(column)) { + hashColumns.add(slot.getExprId()); + } + } + } + StorageBucketHashType function = StorageBucketHashType.STORAGE_BUCKET_CRC32; + if (hmsExternalTable.isBucketedTable()) { + function = StorageBucketHashType.STORAGE_BUCKET_SPARK_MURMUR32; + } + return new DistributionSpecHash(hashColumns, DistributionSpecHash.ShuffleType.NATURAL, + fileScan.getTable().getId(), -1, Collections.emptySet(), function); + } + + return DistributionSpecStorageAny.INSTANCE; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/DataPartition.java b/fe/fe-core/src/main/java/org/apache/doris/planner/DataPartition.java index ce57a57c37780a..27877311aa2d0c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/DataPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/DataPartition.java @@ -26,6 +26,7 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.thrift.TDataPartition; import org.apache.doris.thrift.TExplainLevel; +import org.apache.doris.thrift.THashType; import org.apache.doris.thrift.TPartitionType; import com.google.common.base.Joiner; @@ -50,10 +51,16 @@ public class DataPartition { public static final DataPartition TABLET_ID = new DataPartition(TPartitionType.TABLET_SINK_SHUFFLE_PARTITIONED); private final TPartitionType type; + private final THashType hashType; + // for hash partition: exprs used to compute hash value private ImmutableList partitionExprs; public DataPartition(TPartitionType type, List exprs) { + this(type, exprs, THashType.CRC32); + } + + public DataPartition(TPartitionType type, List exprs, THashType hashType) { Preconditions.checkNotNull(exprs); Preconditions.checkState(!exprs.isEmpty()); Preconditions.checkState(type == TPartitionType.HASH_PARTITIONED @@ -62,6 +69,7 @@ public DataPartition(TPartitionType type, List exprs) { || type == TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED); this.type = type; this.partitionExprs = ImmutableList.copyOf(exprs); + this.hashType = hashType; } public DataPartition(TPartitionType type) { @@ -71,10 +79,15 @@ public DataPartition(TPartitionType type) { || type == TPartitionType.TABLET_SINK_SHUFFLE_PARTITIONED); this.type = type; this.partitionExprs = ImmutableList.of(); + this.hashType = THashType.CRC32; + } + + public static DataPartition hashPartitioned(List exprs, THashType hashType) { + return new DataPartition(TPartitionType.HASH_PARTITIONED, exprs, hashType); } public static DataPartition hashPartitioned(List exprs) { - return new DataPartition(TPartitionType.HASH_PARTITIONED, exprs); + return new DataPartition(TPartitionType.HASH_PARTITIONED, exprs, THashType.CRC32); } public void substitute(ExprSubstitutionMap smap, Analyzer analyzer) throws AnalysisException { @@ -102,17 +115,25 @@ public List getPartitionExprs() { return partitionExprs; } + public THashType getHashType() { + return hashType; + } + public TDataPartition toThrift() { TDataPartition result = new TDataPartition(type); if (partitionExprs != null) { result.setPartitionExprs(Expr.treesToThrift(partitionExprs)); } + result.setHashType(hashType); return result; } public String getExplainString(TExplainLevel explainLevel) { StringBuilder str = new StringBuilder(); str.append(type.toString()); + if (type == TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED) { + str.append("(").append(hashType.toString()).append(")"); + } if (explainLevel == TExplainLevel.BRIEF) { return str.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java index e1a8d36424eebe..4cc9608088cb81 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java @@ -33,17 +33,22 @@ import org.apache.doris.catalog.DistributionInfo; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.HashDistributionInfo; +import org.apache.doris.catalog.HiveExternalDistributionInfo; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Table; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; +import org.apache.doris.datasource.hive.HMSExternalTable; +import org.apache.doris.datasource.hive.source.HiveScanNode; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.thrift.THashType; import org.apache.doris.thrift.TPartitionType; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import org.apache.hive.common.util.Ref; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -285,6 +290,10 @@ private PlanFragment createScanFragment(PlanNode node) throws UserException { OlapScanNode olapScanNode = (OlapScanNode) node; return new PlanFragment(ctx.getNextFragmentId(), node, olapScanNode.constructInputPartitionByDistributionInfo(), DataPartition.RANDOM); + } else if (node instanceof HiveScanNode) { + HiveScanNode hiveScanNode = (HiveScanNode) node; + return new PlanFragment(ctx.getNextFragmentId(), node, + hiveScanNode.constructInputPartitionByDistributionInfo(), DataPartition.RANDOM); } else { // other scan nodes are random partitioned: es, broker return new PlanFragment(ctx.getNextFragmentId(), node, DataPartition.RANDOM); @@ -327,10 +336,12 @@ private PlanFragment createHashJoinFragment( // bucket shuffle join is better than broadcast and shuffle join // it can reduce the network cost of join, so doris chose it first List rhsPartitionExprs = Lists.newArrayList(); - if (canBucketShuffleJoin(node, leftChildFragment, rhsPartitionExprs)) { + Ref hashType = Ref.from(THashType.CRC32); + if (canBucketShuffleJoin(node, leftChildFragment, rhsPartitionExprs, hashType)) { node.setDistributionMode(HashJoinNode.DistributionMode.BUCKET_SHUFFLE); DataPartition rhsJoinPartition = - new DataPartition(TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED, rhsPartitionExprs); + new DataPartition(TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED, + rhsPartitionExprs, hashType.value); ExchangeNode rhsExchange = new ExchangeNode(ctx.getNextNodeId(), rightChildFragment.getPlanRoot(), false); rhsExchange.setNumInstances(rightChildFragment.getPlanRoot().getNumInstances()); @@ -600,7 +611,7 @@ private boolean dataDistributionMatchEqPredicate(List eqJoinPre } private boolean canBucketShuffleJoin(HashJoinNode node, PlanFragment leftChildFragment, - List rhsHashExprs) { + List rhsHashExprs, Ref hashType) { if (node.getJoinOp() == JoinOperator.NULL_AWARE_LEFT_ANTI_JOIN) { return false; } @@ -616,7 +627,9 @@ private boolean canBucketShuffleJoin(HashJoinNode node, PlanFragment leftChildFr PlanNode leftRoot = leftChildFragment.getPlanRoot(); // 1.leftRoot be OlapScanNode if (leftRoot instanceof OlapScanNode) { - return canBucketShuffleJoin(node, leftRoot, rhsHashExprs); + return canBucketShuffleJoin(node, (OlapScanNode) leftRoot, rhsHashExprs); + } else if (leftRoot instanceof HiveScanNode) { + return canBucketShuffleJoin(node, (HiveScanNode) leftRoot, rhsHashExprs, hashType); } // 2.leftRoot be hashjoin node @@ -625,17 +638,83 @@ private boolean canBucketShuffleJoin(HashJoinNode node, PlanFragment leftChildFr leftRoot = leftRoot.getChild(0); } if (leftRoot instanceof OlapScanNode) { - return canBucketShuffleJoin(node, leftRoot, rhsHashExprs); + return canBucketShuffleJoin(node, (OlapScanNode) leftRoot, rhsHashExprs); + } else if (leftRoot instanceof HiveScanNode) { + return canBucketShuffleJoin(node, (HiveScanNode) leftRoot, rhsHashExprs, hashType); } } return false; } + private boolean canBucketShuffleJoin(HashJoinNode node, HiveScanNode leftScanNode, + List rhsJoinExprs, Ref hashType) { + HMSExternalTable leftTable = leftScanNode.getHiveTable(); + + DistributionInfo leftDistribution = leftTable.getDefaultDistributionInfo(); + if (leftDistribution == null || !(leftDistribution instanceof HiveExternalDistributionInfo)) { + return false; + } + + HiveExternalDistributionInfo hiveDistributionInfo = (HiveExternalDistributionInfo) leftDistribution; + + List leftDistributeColumns = hiveDistributionInfo.getDistributionColumns(); + List leftDistributeColumnNames = leftDistributeColumns.stream() + .map(col -> leftTable.getName() + "." + col.getName().toLowerCase()).collect(Collectors.toList()); + + List leftJoinColumnNames = new ArrayList<>(); + List rightExprs = new ArrayList<>(); + List eqJoinConjuncts = node.getEqJoinConjuncts(); + + for (BinaryPredicate eqJoinPredicate : eqJoinConjuncts) { + Expr lhsJoinExpr = eqJoinPredicate.getChild(0); + Expr rhsJoinExpr = eqJoinPredicate.getChild(1); + if (lhsJoinExpr.unwrapSlotRef() == null || rhsJoinExpr.unwrapSlotRef() == null) { + continue; + } + + SlotRef leftSlot = node.getChild(0).findSrcSlotRef(lhsJoinExpr.unwrapSlotRef()); + if (leftSlot.getTable() instanceof HMSExternalTable + && leftScanNode.desc.getSlots().contains(leftSlot.getDesc())) { + // table name in SlotRef is not the really name. `select * from test as t` + // table name in SlotRef is `t`, but here we need is `test`. + leftJoinColumnNames.add(leftSlot.getTable().getName() + "." + + leftSlot.getColumnName().toLowerCase()); + rightExprs.add(rhsJoinExpr); + } + } + + //2 the join columns should contains all left table distribute columns to enable bucket shuffle join + for (int i = 0; i < leftDistributeColumnNames.size(); i++) { + String distributeColumnName = leftDistributeColumnNames.get(i); + boolean findRhsExprs = false; + // check the join column name is same as distribute column name and + // check the rhs join expr type is same as distribute column + for (int j = 0; j < leftJoinColumnNames.size(); j++) { + if (leftJoinColumnNames.get(j).equals(distributeColumnName)) { + // varchar and string type don't need to check the length property + if ((rightExprs.get(j).getType().isVarcharOrStringType() + && leftDistributeColumns.get(i).getType().isVarcharOrStringType()) + || (rightExprs.get(j).getType().equals(leftDistributeColumns.get(i).getType()))) { + rhsJoinExprs.add(rightExprs.get(j)); + findRhsExprs = true; + break; + } + } + } + + if (!findRhsExprs) { + return false; + } + } + + hashType.value = leftScanNode.getHashType(); + return true; + } + //the join expr must contian left table distribute column - private boolean canBucketShuffleJoin(HashJoinNode node, PlanNode leftRoot, + private boolean canBucketShuffleJoin(HashJoinNode node, OlapScanNode leftScanNode, List rhsJoinExprs) { - OlapScanNode leftScanNode = ((OlapScanNode) leftRoot); OlapTable leftTable = leftScanNode.getOlapTable(); //1 the left table has more than one partition or left table is not a stable colocate table diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java index 92175523f227a6..cf18613d85ded6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java @@ -79,6 +79,7 @@ import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TExplainLevel; import org.apache.doris.thrift.TExpr; +import org.apache.doris.thrift.THashType; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TNormalizedOlapScanNode; import org.apache.doris.thrift.TNormalizedPlanNode; @@ -1860,7 +1861,7 @@ public DataPartition constructInputPartitionByDistributionInfo() throws UserExce SlotRef slotRef = new SlotRef(desc.getRef().getName(), column.getName()); dataDistributeExprs.add(slotRef); } - return DataPartition.hashPartitioned(dataDistributeExprs); + return DataPartition.hashPartitioned(dataDistributeExprs, THashType.CRC32); } else { return DataPartition.RANDOM; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 472f2462e4d39d..2a078977ede498 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -24,6 +24,7 @@ import org.apache.doris.catalog.FsBroker; import org.apache.doris.common.Config; import org.apache.doris.common.MarkedCountDownLatch; +import org.apache.doris.common.NotImplementedException; import org.apache.doris.common.Pair; import org.apache.doris.common.Reference; import org.apache.doris.common.Status; @@ -36,6 +37,7 @@ import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.ExternalScanNode; import org.apache.doris.datasource.FileQueryScanNode; +import org.apache.doris.datasource.hive.source.HiveScanNode; import org.apache.doris.datasource.hive.HMSTransaction; import org.apache.doris.datasource.iceberg.IcebergTransaction; import org.apache.doris.load.loadv2.LoadJob; @@ -2167,8 +2169,13 @@ protected void computeScanRangeAssignment() throws Exception { replicaNumPerHost, isEnableOrderedLocations); } if (fragmentContainsBucketShuffleJoin) { - bucketShuffleJoinController.computeScanRangeAssignmentByBucket((OlapScanNode) scanNode, - idToBackend, addressToBackendID, replicaNumPerHost); + if (scanNode instanceof OlapScanNode) { + bucketShuffleJoinController.computeScanRangeAssignmentByBucket((OlapScanNode) scanNode, + idToBackend, addressToBackendID, replicaNumPerHost); + } else if (scanNode instanceof HiveScanNode) { + bucketShuffleJoinController.computeScanRangeAssignmentByBucket((HiveScanNode) scanNode, + idToBackend, addressToBackendID, replicaNumPerHost); + } } if (!(fragmentContainsColocateJoin || fragmentContainsBucketShuffleJoin)) { computeScanRangeAssignmentByScheduler(scanNode, locations, assignment, assignedBytesPerHost, @@ -2687,6 +2694,50 @@ private void computeScanRangeAssignmentByBucket( } } + private void computeScanRangeAssignmentByBucket( + final HiveScanNode scanNode, ImmutableMap idToBackend, + Map addressToBackendID, + Map replicaNumPerHost) throws Exception { + if (!fragmentIdToSeqToAddressMap.containsKey(scanNode.getFragmentId())) { + int bucketNum = 0; + if (scanNode.getHiveTable().isBucketedTable()) { + bucketNum = scanNode.getHiveTable().getDefaultDistributionInfo().getBucketNum(); + } else { + throw new NotImplementedException("bucket shuffle for non-bucketed table not supported"); + } + fragmentIdToBucketNumMap.put(scanNode.getFragmentId(), bucketNum); + fragmentIdToSeqToAddressMap.put(scanNode.getFragmentId(), new HashMap<>()); + fragmentIdBucketSeqToScanRangeMap.put(scanNode.getFragmentId(), new BucketSeqToScanRange()); + fragmentIdToBuckendIdBucketCountMap.put(scanNode.getFragmentId(), new HashMap<>()); + } + Map bucketSeqToAddress + = fragmentIdToSeqToAddressMap.get(scanNode.getFragmentId()); + BucketSeqToScanRange bucketSeqToScanRange = fragmentIdBucketSeqToScanRangeMap.get(scanNode.getFragmentId()); + + for (Integer bucketSeq : scanNode.bucketSeq2locations.keySet()) { + //fill scanRangeParamsList + List locations = scanNode.bucketSeq2locations.get(bucketSeq); + if (!bucketSeqToAddress.containsKey(bucketSeq)) { + getExecHostPortForFragmentIDAndBucketSeq(locations.get(0), scanNode.getFragmentId(), + bucketSeq, idToBackend, addressToBackendID, replicaNumPerHost); + } + + for (TScanRangeLocations location : locations) { + Map> scanRanges = + findOrInsert(bucketSeqToScanRange, bucketSeq, new HashMap<>()); + + List scanRangeParamsList = + findOrInsert(scanRanges, scanNode.getId().asInt(), new ArrayList<>()); + + // add scan range + TScanRangeParams scanRangeParams = new TScanRangeParams(); + scanRangeParams.scan_range = location.scan_range; + scanRangeParamsList.add(scanRangeParams); + updateScanRangeNumByScanRange(scanRangeParams); + } + } + } + private void computeInstanceParam(PlanFragmentId fragmentId, int parallelExecInstanceNum, FragmentExecParams params, boolean hasNullAwareLeftAntiJoin) { assignScanRanges(fragmentId, parallelExecInstanceNum, params, fragmentIdBucketSeqToScanRangeMap, diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/CoordinatorTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/CoordinatorTest.java index b6632a39db71d6..eb1bfbb0c25490 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/CoordinatorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/CoordinatorTest.java @@ -41,6 +41,7 @@ import org.apache.doris.planner.ScanNode; import org.apache.doris.service.FrontendOptions; import org.apache.doris.system.Backend; +import org.apache.doris.thrift.THashType; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TPartitionType; import org.apache.doris.thrift.TScanRangeLocation; @@ -175,7 +176,7 @@ public void testIsBucketShuffleJoin() { new ArrayList<>()); hashJoinNode.setFragment(new PlanFragment(new PlanFragmentId(-1), hashJoinNode, - new DataPartition(TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED, testJoinexprs))); + new DataPartition(TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED, testJoinexprs, THashType.CRC32))); // hash join node is not bucket shuffle join Assert.assertEquals(false, @@ -183,13 +184,13 @@ public void testIsBucketShuffleJoin() { // the fragment id is different from hash join node hashJoinNode.setFragment(new PlanFragment(new PlanFragmentId(-2), hashJoinNode, - new DataPartition(TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED, testJoinexprs))); + new DataPartition(TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED, testJoinexprs, THashType.CRC32))); hashJoinNode.setDistributionMode(HashJoinNode.DistributionMode.BUCKET_SHUFFLE); Assert.assertEquals(false, Deencapsulation.invoke(bucketShuffleJoinController, "isBucketShuffleJoin", -1, hashJoinNode)); hashJoinNode.setFragment(new PlanFragment(new PlanFragmentId(-1), hashJoinNode, - new DataPartition(TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED, testJoinexprs))); + new DataPartition(TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED, testJoinexprs, THashType.CRC32))); Assert.assertEquals(true, Deencapsulation.invoke(bucketShuffleJoinController, "isBucketShuffleJoin", -1, hashJoinNode)); diff --git a/gensrc/thrift/Ddl.thrift b/gensrc/thrift/Ddl.thrift index 9696230af909ed..f733637bc7791a 100644 --- a/gensrc/thrift/Ddl.thrift +++ b/gensrc/thrift/Ddl.thrift @@ -76,10 +76,6 @@ enum TAggType { // 4: optional string default_value //} -enum THashType { - CRC32 -} - // random partition info struct TRandomPartitionDesc { } @@ -93,7 +89,7 @@ struct THashPartitionDesc { 2: required i32 hash_buckets // type to compute hash value. if not set, use CRC32 - 3: optional THashType hash_type + 3: optional Partitions.THashType hash_type } // value used to represents one column value in one range value diff --git a/gensrc/thrift/Partitions.thrift b/gensrc/thrift/Partitions.thrift index 4e306c2970bd43..f942981ad6697a 100644 --- a/gensrc/thrift/Partitions.thrift +++ b/gensrc/thrift/Partitions.thrift @@ -21,6 +21,12 @@ namespace java org.apache.doris.thrift include "Exprs.thrift" include "Types.thrift" +enum THashType { + CRC32, + XXHASH64, + SPARK_MURMUR32 +} + enum TPartitionType { UNPARTITIONED, @@ -96,6 +102,7 @@ struct TDataPartition { 1: required TPartitionType type 2: optional list partition_exprs 3: optional list partition_infos + 4: optional THashType hash_type } diff --git a/regression-test/data/external_table_p0/hive/test_hive_spark_clustered_table.out b/regression-test/data/external_table_p0/hive/test_hive_spark_clustered_table.out new file mode 100644 index 00000000000000..23f03f24aea961 --- /dev/null +++ b/regression-test/data/external_table_p0/hive/test_hive_spark_clustered_table.out @@ -0,0 +1,158 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q01 -- +1 U1 IN 1 U1 IN +11 U11 IN 11 U11 IN +21 U21 IN 21 U21 IN +31 U31 IN 31 U31 IN + +-- !q02 -- +PLAN FRAGMENT 0 + OUTPUT EXPRS: + user_id[#12] + key[#13] + part[#14] + user_id[#15] + key[#16] + part[#17] + PARTITION: UNPARTITIONED + + HAS_COLO_PLAN_NODE: false + + VRESULT SINK + MYSQL_PROTOCAL + + 4:VEXCHANGE + offset: 0 + distribute expr lists: user_id[#12] + +PLAN FRAGMENT 1 + + PARTITION: HASH_PARTITIONED: user_id[#3] + + HAS_COLO_PLAN_NODE: false + + STREAM DATA SINK + EXCHANGE ID: 04 + UNPARTITIONED + + 3:VHASH JOIN(165) + | join op: INNER JOIN(BUCKET_SHUFFLE)[] + | equal join conjunct: (user_id[#3] = user_id[#0]) + | cardinality=143 + | vec output tuple id: 3 + | vIntermediate tuple ids: 2 + | hash output slot ids: 0 1 2 3 4 5 + | distribute expr lists: user_id[#3] + | distribute expr lists: user_id[#0] + | + |----1:VEXCHANGE + | offset: 0 + | distribute expr lists: user_id[#0] + | + 2:VHIVE_SCAN_NODE(158) + table: parquet_test2 + inputSplitNum=4, totalFileSize=2873, scanRanges=4 + partition=1/1 + cardinality=143, numNodes=1 + pushdown agg=NONE + +PLAN FRAGMENT 2 + + PARTITION: HASH_PARTITIONED: user_id[#0] + + HAS_COLO_PLAN_NODE: false + + STREAM DATA SINK + EXCHANGE ID: 01 + BUCKET_SHFFULE_HASH_PARTITIONED(SPARK_MURMUR32): user_id[#0] + + 0:VHIVE_SCAN_NODE(159) + table: parquet_test2 + inputSplitNum=4, totalFileSize=2873, scanRanges=4 + partition=1/1 + cardinality=143, numNodes=1 + pushdown agg=NONE + +-- !q03 -- +1 U1 IN 1 U1 IN +11 U11 IN 11 U11 IN +21 U21 IN 21 U21 IN +31 U31 IN 31 U31 IN + +-- !q01 -- +1 U1 IN 1 U1 IN +11 U11 IN 11 U11 IN +21 U21 IN 21 U21 IN +31 U31 IN 31 U31 IN + +-- !q02 -- +PLAN FRAGMENT 0 + OUTPUT EXPRS: + + + + + + + PARTITION: UNPARTITIONED + + HAS_COLO_PLAN_NODE: false + + VRESULT SINK + MYSQL_PROTOCAL + + 4:VEXCHANGE + offset: 0 + +PLAN FRAGMENT 1 + + PARTITION: HASH_PARTITIONED: `hive_test_parquet`.`default`.`parquet_test2`.`user_id` + + HAS_COLO_PLAN_NODE: false + + STREAM DATA SINK + EXCHANGE ID: 04 + UNPARTITIONED + + 2:VHASH JOIN + | join op: INNER JOIN(BUCKET_SHUFFLE)[Only olap table support colocate plan] + | equal join conjunct: (`t1`.`user_id` = `t2`.`user_id`) + | cardinality=-1 + | vec output tuple id: 2 + | vIntermediate tuple ids: 3 4 + | output slot ids: 6 7 8 9 10 11 + | hash output slot ids: 0 1 2 3 4 5 + | + |----3:VEXCHANGE + | offset: 0 + | + 0:VHIVE_SCAN_NODE + table: parquet_test2 + inputSplitNum=4, totalFileSize=2873, scanRanges=4 + partition=1/1 + numNodes=1 + pushdown agg=NONE + +PLAN FRAGMENT 2 + + PARTITION: HASH_PARTITIONED: `hive_test_parquet`.`default`.`parquet_test2`.`user_id` + + HAS_COLO_PLAN_NODE: false + + STREAM DATA SINK + EXCHANGE ID: 03 + BUCKET_SHFFULE_HASH_PARTITIONED(SPARK_MURMUR32): `t2`.`user_id` + + 1:VHIVE_SCAN_NODE + table: parquet_test2 + inputSplitNum=4, totalFileSize=2873, scanRanges=4 + partition=1/1 + numNodes=1 + pushdown agg=NONE + +-- !q03 -- +1 U1 IN 1 U1 IN +11 U11 IN 11 U11 IN +21 U21 IN 21 U21 IN +31 U31 IN 31 U31 IN + diff --git a/regression-test/suites/external_table_p0/hive/test_hive_spark_clustered_table.groovy b/regression-test/suites/external_table_p0/hive/test_hive_spark_clustered_table.groovy new file mode 100644 index 00000000000000..bf7f5c1794a96f --- /dev/null +++ b/regression-test/suites/external_table_p0/hive/test_hive_spark_clustered_table.groovy @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_hive_spark_clustered_table", "p0,external,hive,external_docker,external_docker_hive") { + def q01 = { + qt_q01 """ select * from parquet_test2 t1, parquet_test2 t2 WHERE t1.user_id = t2.user_id ORDER BY 1,2 ;""" + + qt_q02 """explain select * from parquet_test2 t1, parquet_test2 t2 WHERE t1.user_id = t2.user_id ;""" + + qt_q03 """select * from parquet_test2 t1, `internal`.`regression_test`.doris_dist_test t2 WHERE t1.user_id = t2.user_id ORDER BY 1,2 ;""" + + explain { + sql("""select * from parquet_test2 t1, `internal`.`regression_test`.doris_dist_test t2 WHERE t1.user_id = t2.user_id;""") + contains "join op: INNER JOIN(BUCKET_SHUFFLE)" + contains "BUCKET_SHFFULE_HASH_PARTITIONED(SPARK_MURMUR32)" + } + } + + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + try { + String hms_port = context.config.otherConfigs.get("hms_port") + String catalog_name = "hive_test_parquet" + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + sql """drop catalog if exists ${catalog_name}""" + sql """create catalog if not exists ${catalog_name} properties ( + "type"="hms", + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}' + );""" + + sql """use `regression_test`""" + sql """drop table if exists doris_dist_test;""" + sql """create table doris_dist_test properties("replication_num"="1") + as select * from `${catalog_name}`.`default`.parquet_test2; """ + + sql """use `${catalog_name}`.`default`""" + + sql """set enable_fallback_to_original_planner=false;""" + + q01() + + sql """set enable_nereids_planner=false;""" + + q01() + + sql """use `internal`.`regression_test`""" + sql """drop table if exists doris_dist_test; """ + sql """drop catalog if exists ${catalog_name}; """ + } finally { + } + } +} diff --git a/regression-test/suites/nereids_p0/join/bucket_shuffle_join.groovy b/regression-test/suites/nereids_p0/join/bucket_shuffle_join.groovy new file mode 100644 index 00000000000000..dd6a00b3123e74 --- /dev/null +++ b/regression-test/suites/nereids_p0/join/bucket_shuffle_join.groovy @@ -0,0 +1,83 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("bucket-shuffle-join") { + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql 'SET be_number_for_test=1' + sql 'SET parallel_pipeline_task_num=1' + order_qt_test_bucket """ + select * from test_bucket_shuffle_join where rectime="2021-12-01 00:00:00" and id in (select k1 from test_join where k1 in (1,2)) + """ + + sql """ DROP TABLE IF EXISTS shuffle_join_t1 """ + sql """ DROP TABLE IF EXISTS shuffle_join_t2 """ + + sql """ + create table shuffle_join_t1 ( a varchar(10) not null ) + ENGINE=OLAP + DISTRIBUTED BY HASH(a) BUCKETS 5 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "in_memory" = "false", + "storage_format" = "V2" + ); + """ + + sql """ + create table shuffle_join_t2 ( a varchar(5) not null, b string not null, c char(3) not null ) + ENGINE=OLAP + DISTRIBUTED BY HASH(a) BUCKETS 5 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "in_memory" = "false", + "storage_format" = "V2" + ); + """ + + sql """insert into shuffle_join_t1 values("1");""" + sql """insert into shuffle_join_t1 values("1");""" + sql """insert into shuffle_join_t1 values("1");""" + sql """insert into shuffle_join_t1 values("1");""" + sql """insert into shuffle_join_t2 values("1","1","1");""" + sql """insert into shuffle_join_t2 values("1","1","1");""" + sql """insert into shuffle_join_t2 values("1","1","1");""" + sql """insert into shuffle_join_t2 values("1","1","1");""" + + sql """analyze table shuffle_join_t1 with sync;""" + sql """analyze table shuffle_join_t2 with sync;""" + + // we must disable join reorder since right xx join cannot be bucket shuffle join now + sql """set disable_join_reorder=true""" + + explain { + sql("select * from shuffle_join_t1 t1 left join shuffle_join_t2 t2 on t1.a = t2.a;") + contains "BUCKET_SHUFFLE" + } + + explain { + sql("select * from shuffle_join_t1 t1 left join shuffle_join_t2 t2 on t1.a = t2.b;") + contains "BUCKET_SHUFFLE" + } + + explain { + sql("select * from shuffle_join_t1 t1 left join shuffle_join_t2 t2 on t1.a = t2.c;") + contains "BUCKET_SHUFFLE" + contains "BUCKET_SHFFULE_HASH_PARTITIONED(CRC32): c" + } + +} From 7fdb0a6f5e4875fff034ddfbcc1e04ad61bb584b Mon Sep 17 00:00:00 2001 From: morningman Date: Fri, 1 Mar 2024 16:27:26 +0800 Subject: [PATCH 2/3] [opt] refine some method name --- .../catalog/HiveExternalDistributionInfo.java | 15 ---------- .../datasource/FederationBackendPolicy.java | 2 -- .../doris/datasource/FileQueryScanNode.java | 8 +++--- .../datasource/hive/HMSExternalTable.java | 17 +++++------ .../doris/datasource/hive/HiveBucketUtil.java | 6 ++++ .../datasource/hive/source/HiveScanNode.java | 4 +-- .../translator/PhysicalPlanTranslator.java | 12 +------- .../properties/DistributionSpecHash.java | 18 +++++++++++- .../LogicalFileScanToPhysicalFileScan.java | 2 +- .../doris/planner/DistributedPlanner.java | 28 +++++++++++-------- .../java/org/apache/doris/qe/Coordinator.java | 28 ++++++++++++------- 11 files changed, 73 insertions(+), 67 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveExternalDistributionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveExternalDistributionInfo.java index 5b15874401908a..d30d0f2e36cbfb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveExternalDistributionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveExternalDistributionInfo.java @@ -29,26 +29,11 @@ public class HiveExternalDistributionInfo extends HashDistributionInfo { @SerializedName(value = "bucketingVersion") private final int bucketingVersion; - public HiveExternalDistributionInfo() { - bucketingVersion = 2; - } - public HiveExternalDistributionInfo(int bucketNum, List distributionColumns, int bucketingVersion) { super(bucketNum, distributionColumns); this.bucketingVersion = bucketingVersion; } - public HiveExternalDistributionInfo(int bucketNum, boolean autoBucket, - List distributionColumns, int bucketingVersion) { - super(bucketNum, autoBucket, distributionColumns); - this.bucketingVersion = bucketingVersion; - } - - public int getBucketingVersion() { - return bucketingVersion; - } - - @Override public boolean equals(Object o) { if (this == o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/FederationBackendPolicy.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/FederationBackendPolicy.java index a7f6b9431e0104..639a2bd715f7ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/FederationBackendPolicy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/FederationBackendPolicy.java @@ -203,8 +203,6 @@ public void init(BeSelectionPolicy policy) throws UserException { } catch (ExecutionException e) { throw new UserException("failed to get consistent hash", e); } - /*consistentBucket = new ConsistentHash<>(Hashing.murmur3_128(), new BucketHash(), - new BackendHash(), backends, Config.virtual_node_number);*/ } public Backend getNextBe() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java index c41c6e6aabd342..caebca70cf57c0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java @@ -372,12 +372,12 @@ public void createScanRangeLocations() throws UserException { ? BrokerUtil.parseColumnsFromPath(fileSplit.getPath().toString(), pathPartitionKeys, false, isACID) : fileSplit.getPartitionValues(); - boolean isBucketedHiveTable = false; + boolean isSparkBucketedHiveTable = false; int bucketNum = 0; TableIf targetTable = getTargetTable(); if (targetTable instanceof HMSExternalTable) { - isBucketedHiveTable = ((HMSExternalTable) targetTable).isBucketedTable(); - if (isBucketedHiveTable) { + isSparkBucketedHiveTable = ((HMSExternalTable) targetTable).isSparkBucketedTable(); + if (isSparkBucketedHiveTable) { bucketNum = HiveBucketUtil.getBucketNumberFromPath(fileSplit.getPath().getName()).getAsInt(); } } @@ -425,7 +425,7 @@ >>>>>>> a5ce2395a2 ([feature](datalake) Add BucketShuffleJoin support for Hive t fileSplit.getStart(), fileSplit.getLength(), Joiner.on("|").join(fileSplit.getHosts())); } - if (isBucketedHiveTable) { + if (isSparkBucketedHiveTable) { bucketSeq2locations.put(bucketNum, curLocations); } >>>>>>> a5ce2395a2 ([feature](datalake) Add BucketShuffleJoin support for Hive table data generated by Spark. (27783)) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index c3757ef5a6593e..aa49d7d601ed8f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -38,6 +38,7 @@ import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.SchemaCacheValue; import org.apache.doris.datasource.TablePartitionValues; +import org.apache.doris.datasource.hive.HiveBucketUtil.HiveBucketType; import org.apache.doris.datasource.hudi.HudiSchemaCacheValue; import org.apache.doris.datasource.hudi.HudiUtils; import org.apache.doris.datasource.iceberg.IcebergUtils; @@ -80,7 +81,6 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.StringColumnStatsData; import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -180,7 +180,7 @@ public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableI protected volatile org.apache.hadoop.hive.metastore.api.Table remoteTable = null; protected List partitionColumns; private List bucketColumns; - private boolean isSparkTable; + private HiveBucketType hiveBucketType = HiveBucketType.NONE; private DLAType dlaType = DLAType.UNKNOWN; @@ -272,12 +272,8 @@ public boolean isHoodieCowTable() { || (params != null && "COPY_ON_WRITE".equalsIgnoreCase(params.get("flink.table.type"))); } - public boolean isSparkTable() { - return isSparkTable; - } - - public boolean isBucketedTable() { - return bucketColumns != null && !bucketColumns.isEmpty() && isSparkTable; + public boolean isSparkBucketedTable() { + return bucketColumns != null && !bucketColumns.isEmpty() && hiveBucketType == HiveBucketType.SPARK; } /** @@ -589,7 +585,7 @@ private Optional getIcebergSchema() { private void initBucketingColumns(List columns) { List bucketCols = new ArrayList<>(5); int numBuckets = getBucketColumns(bucketCols); - if (bucketCols.isEmpty() || !isSparkTable) { + if (bucketCols.isEmpty() || hiveBucketType != HiveBucketType.SPARK) { bucketColumns = ImmutableList.of(); distributionInfo = new RandomDistributionInfo(1, true); return; @@ -626,6 +622,7 @@ private int getBucketColumns(List bucketCols) { /* Hive Bucketed Table */ bucketCols.addAll(descriptor.getBucketCols()); numBuckets = descriptor.getNumBuckets(); + hiveBucketType = HiveBucketType.HIVE; } else if (remoteTable.isSetParameters() && !Collections.disjoint(SUPPORTED_BUCKET_PROPERTIES, remoteTable.getParameters().keySet())) { Map parameters = remoteTable.getParameters(); @@ -640,7 +637,7 @@ private int getBucketColumns(List bucketCols) { } if (numBuckets > 0) { - isSparkTable = true; + hiveBucketType = HiveBucketType.SPARK; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveBucketUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveBucketUtil.java index ce0d9cfba98bf7..fc0bed8d5e1c25 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveBucketUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveBucketUtil.java @@ -65,6 +65,12 @@ public class HiveBucketUtil { private static final Logger LOG = LogManager.getLogger(HiveBucketUtil.class); + public enum HiveBucketType { + NONE, + HIVE, + SPARK + } + private static final Set SUPPORTED_TYPES_FOR_BUCKET_FILTER = ImmutableSet.of( PrimitiveType.BOOLEAN, PrimitiveType.TINYINT, diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java index 527c141da8d10e..e1d65125908ef7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java @@ -507,7 +507,7 @@ protected TFileCompressType getFileCompressType(FileSplit fileSplit) throws User @Override public DataPartition constructInputPartitionByDistributionInfo() { - if (hmsTable.isBucketedTable()) { + if (hmsTable.isSparkBucketedTable()) { DistributionInfo distributionInfo = hmsTable.getDefaultDistributionInfo(); if (!(distributionInfo instanceof HashDistributionInfo)) { return DataPartition.RANDOM; @@ -530,7 +530,7 @@ public HMSExternalTable getHiveTable() { @Override public THashType getHashType() { - if (hmsTable.isBucketedTable() + if (hmsTable.isSparkBucketedTable() && hmsTable.getDefaultDistributionInfo() instanceof HashDistributionInfo) { return THashType.SPARK_MURMUR32; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 202a203544fd9d..b44ea8e5631638 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -2644,17 +2644,7 @@ private DataPartition toDataPartition(DistributionSpec distributionSpec, THashType hashType = THashType.XXHASH64; switch (distributionSpecHash.getShuffleType()) { case STORAGE_BUCKETED: - switch (distributionSpecHash.getShuffleFunction()) { - case STORAGE_BUCKET_SPARK_MURMUR32: - hashType = THashType.SPARK_MURMUR32; - break; - case STORAGE_BUCKET_CRC32: - hashType = THashType.CRC32; - break; - case STORAGE_BUCKET_XXHASH64: - default: - break; - } + hashType = distributionSpecHash.getShuffleFunction().toThrift(); partitionType = TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED; break; case EXECUTION_BUCKETED: diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java index 5bf1a7f52472bc..3bd1ab1ae52964 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java @@ -20,6 +20,7 @@ import org.apache.doris.nereids.annotation.Developing; import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.util.Utils; +import org.apache.doris.thrift.THashType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -338,9 +339,24 @@ public enum ShuffleType { * Enums for concrete shuffle functions. */ public enum StorageBucketHashType { + // CRC32 is for Doris internal storage bucket hash function STORAGE_BUCKET_CRC32, + // XXHASH64 is the default hash function for Doris computation layer STORAGE_BUCKET_XXHASH64, - STORAGE_BUCKET_SPARK_MURMUR32 + // SPARK_MURMUR32 is the hash function for Spark bucketed hive table's storage and computation + STORAGE_BUCKET_SPARK_MURMUR32; + + public THashType toThrift() { + switch (this) { + case STORAGE_BUCKET_CRC32: + return THashType.CRC32; + case STORAGE_BUCKET_SPARK_MURMUR32: + return THashType.SPARK_MURMUR32; + case STORAGE_BUCKET_XXHASH64: + default: + return THashType.XXHASH64; + } + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java index a0fa806ed2427b..5162949ed796d1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java @@ -81,7 +81,7 @@ private DistributionSpec convertDistribution(LogicalFileScan fileScan) { } } StorageBucketHashType function = StorageBucketHashType.STORAGE_BUCKET_CRC32; - if (hmsExternalTable.isBucketedTable()) { + if (hmsExternalTable.isSparkBucketedTable()) { function = StorageBucketHashType.STORAGE_BUCKET_SPARK_MURMUR32; } return new DistributionSpecHash(hashColumns, DistributionSpecHash.ShuffleType.NATURAL, diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java index 4cc9608088cb81..3ba8efa48a29a0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java @@ -625,11 +625,8 @@ private boolean canBucketShuffleJoin(HashJoinNode node, PlanFragment leftChildFr } PlanNode leftRoot = leftChildFragment.getPlanRoot(); - // 1.leftRoot be OlapScanNode - if (leftRoot instanceof OlapScanNode) { - return canBucketShuffleJoin(node, (OlapScanNode) leftRoot, rhsHashExprs); - } else if (leftRoot instanceof HiveScanNode) { - return canBucketShuffleJoin(node, (HiveScanNode) leftRoot, rhsHashExprs, hashType); + if (leftRoot instanceof ScanNode) { + return canBucketShuffleJoin(node, (ScanNode) leftRoot, rhsHashExprs, hashType); } // 2.leftRoot be hashjoin node @@ -637,17 +634,26 @@ private boolean canBucketShuffleJoin(HashJoinNode node, PlanFragment leftChildFr while (leftRoot instanceof HashJoinNode) { leftRoot = leftRoot.getChild(0); } - if (leftRoot instanceof OlapScanNode) { - return canBucketShuffleJoin(node, (OlapScanNode) leftRoot, rhsHashExprs); - } else if (leftRoot instanceof HiveScanNode) { - return canBucketShuffleJoin(node, (HiveScanNode) leftRoot, rhsHashExprs, hashType); + if (leftRoot instanceof ScanNode) { + canBucketShuffleJoin(node, (ScanNode) leftRoot, rhsHashExprs, hashType); } } return false; } - private boolean canBucketShuffleJoin(HashJoinNode node, HiveScanNode leftScanNode, + private boolean canBucketShuffleJoin(HashJoinNode node, ScanNode leftScanNode, + List rhsJoinExprs, Ref hashType) { + if (leftScanNode instanceof OlapScanNode) { + return canBucketShuffleJoinForOlap(node, (OlapScanNode) leftScanNode, rhsJoinExprs); + } else if (leftScanNode instanceof HiveScanNode) { + return canBucketShuffleJoinForHive(node, (HiveScanNode) leftScanNode, rhsJoinExprs, hashType); + } else { + return false; + } + } + + private boolean canBucketShuffleJoinForHive(HashJoinNode node, HiveScanNode leftScanNode, List rhsJoinExprs, Ref hashType) { HMSExternalTable leftTable = leftScanNode.getHiveTable(); @@ -713,7 +719,7 @@ private boolean canBucketShuffleJoin(HashJoinNode node, HiveScanNode leftScanNod } //the join expr must contian left table distribute column - private boolean canBucketShuffleJoin(HashJoinNode node, OlapScanNode leftScanNode, + private boolean canBucketShuffleJoinForOlap(HashJoinNode node, OlapScanNode leftScanNode, List rhsJoinExprs) { OlapTable leftTable = leftScanNode.getOlapTable(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 2a078977ede498..08c442c5877566 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -2169,13 +2169,8 @@ protected void computeScanRangeAssignment() throws Exception { replicaNumPerHost, isEnableOrderedLocations); } if (fragmentContainsBucketShuffleJoin) { - if (scanNode instanceof OlapScanNode) { - bucketShuffleJoinController.computeScanRangeAssignmentByBucket((OlapScanNode) scanNode, - idToBackend, addressToBackendID, replicaNumPerHost); - } else if (scanNode instanceof HiveScanNode) { - bucketShuffleJoinController.computeScanRangeAssignmentByBucket((HiveScanNode) scanNode, - idToBackend, addressToBackendID, replicaNumPerHost); - } + bucketShuffleJoinController.computeScanRangeAssignmentByBucket(scanNode, + idToBackend, addressToBackendID, replicaNumPerHost); } if (!(fragmentContainsColocateJoin || fragmentContainsBucketShuffleJoin)) { computeScanRangeAssignmentByScheduler(scanNode, locations, assignment, assignedBytesPerHost, @@ -2653,8 +2648,21 @@ private void getExecHostPortForFragmentIDAndBucketSeq(TScanRangeLocations seqLoc this.fragmentIdToSeqToAddressMap.get(fragmentId).put(bucketSeq, execHostPort); } - // to ensure the same bucketSeq tablet to the same execHostPort private void computeScanRangeAssignmentByBucket( + final ScanNode scanNode, ImmutableMap idToBackend, + Map addressToBackendID, + Map replicaNumPerHost) throws Exception { + if (scanNode instanceof OlapScanNode) { + computeScanRangeAssignmentByBucketForOlap((OlapScanNode) scanNode, idToBackend, addressToBackendID, + replicaNumPerHost); + } else if (scanNode instanceof HiveScanNode) { + computeScanRangeAssignmentByBucketForHive((HiveScanNode) scanNode, idToBackend, addressToBackendID, + replicaNumPerHost); + } + } + + // to ensure the same bucketSeq tablet to the same execHostPort + private void computeScanRangeAssignmentByBucketForOlap( final OlapScanNode scanNode, ImmutableMap idToBackend, Map addressToBackendID, Map replicaNumPerHost) throws Exception { @@ -2694,13 +2702,13 @@ private void computeScanRangeAssignmentByBucket( } } - private void computeScanRangeAssignmentByBucket( + private void computeScanRangeAssignmentByBucketForHive( final HiveScanNode scanNode, ImmutableMap idToBackend, Map addressToBackendID, Map replicaNumPerHost) throws Exception { if (!fragmentIdToSeqToAddressMap.containsKey(scanNode.getFragmentId())) { int bucketNum = 0; - if (scanNode.getHiveTable().isBucketedTable()) { + if (scanNode.getHiveTable().isSparkBucketedTable()) { bucketNum = scanNode.getHiveTable().getDefaultDistributionInfo().getBucketNum(); } else { throw new NotImplementedException("bucket shuffle for non-bucketed table not supported"); From 3780f2353548a1b9a662d7b2c585a9a4eb1663f2 Mon Sep 17 00:00:00 2001 From: Nitin Kashyap Date: Fri, 31 May 2024 16:01:36 +0530 Subject: [PATCH 3/3] [opt]fixed partitioners, local_exchange and murmur-hash calculations --- be/src/pipeline/dependency.h | 12 +- .../pipeline/exec/exchange_sink_operator.cpp | 47 +++++- be/src/pipeline/exec/exchange_sink_operator.h | 6 + .../exec/exchange_source_operator.cpp | 7 +- .../pipeline/exec/exchange_source_operator.h | 3 +- .../partitioned_hash_join_sink_operator.cpp | 2 + .../partitioned_hash_join_sink_operator.h | 3 +- be/src/vec/columns/column.h | 20 +-- be/src/vec/columns/column_array.cpp | 3 - be/src/vec/columns/column_decimal.cpp | 8 +- be/src/vec/columns/column_map.cpp | 3 - be/src/vec/columns/column_nullable.cpp | 9 +- be/src/vec/columns/column_string.cpp | 9 +- be/src/vec/columns/column_string.h | 6 +- be/src/vec/columns/column_vector.cpp | 6 +- be/src/vec/columns/column_vector.h | 3 - be/src/vec/runtime/partitioner.cpp | 29 ++-- be/src/vec/runtime/partitioner.h | 49 ++++-- be/src/vec/sink/vdata_stream_sender.h | 7 - .../datasource/FederationBackendPolicy.java | 20 +++ .../doris/datasource/FileQueryScanNode.java | 140 +++++++++--------- .../doris/datasource/SplitAssignment.java | 26 +++- .../doris/datasource/SplitToScanRange.java | 1 + .../datasource/hive/HMSExternalTable.java | 12 +- .../doris/datasource/hive/HiveBucketUtil.java | 2 +- .../datasource/hive/source/HiveScanNode.java | 4 +- .../translator/PhysicalPlanTranslator.java | 12 +- .../ChildOutputPropertyDeriver.java | 8 +- .../ChildrenPropertiesRegulator.java | 2 +- .../properties/DistributionSpecHash.java | 22 ++- .../LogicalFileScanToPhysicalFileScan.java | 23 ++- .../doris/planner/DistributedPlanner.java | 5 +- .../apache/doris/planner/ExchangeNode.java | 9 ++ .../apache/doris/planner/HashJoinNode.java | 9 ++ .../java/org/apache/doris/qe/Coordinator.java | 72 +++++---- .../org/apache/doris/qe/SessionVariable.java | 13 ++ gensrc/thrift/PlanNodes.thrift | 2 + .../test_hive_spark_clustered_table.groovy | 1 + .../join/bucket_shuffle_join.groovy | 1 + 39 files changed, 397 insertions(+), 219 deletions(-) diff --git a/be/src/pipeline/dependency.h b/be/src/pipeline/dependency.h index ecbd49a5647c2e..e96d933bdeb101 100644 --- a/be/src/pipeline/dependency.h +++ b/be/src/pipeline/dependency.h @@ -727,14 +727,22 @@ inline std::string get_exchange_type_name(ExchangeType idx) { } struct DataDistribution { - DataDistribution(ExchangeType type) : distribution_type(type) {} + DataDistribution(ExchangeType type) : distribution_type(type), hash_type(THashType::CRC32) {} DataDistribution(ExchangeType type, const std::vector& partition_exprs_) - : distribution_type(type), partition_exprs(partition_exprs_) {} + : distribution_type(type), + partition_exprs(partition_exprs_), + hash_type(THashType::CRC32) {} + DataDistribution(ExchangeType type, const THashType::type hash_type) + : distribution_type(type), hash_type(hash_type) {} + DataDistribution(ExchangeType type, const std::vector& partition_exprs_, + const THashType::type hash) + : distribution_type(type), partition_exprs(partition_exprs_), hash_type(hash) {} DataDistribution(const DataDistribution& other) = default; bool need_local_exchange() const { return distribution_type != ExchangeType::NOOP; } DataDistribution& operator=(const DataDistribution& other) = default; ExchangeType distribution_type; std::vector partition_exprs; + THashType::type hash_type; }; class ExchangerBase; diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index cc789f6e25b20b..0d350361e12054 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -78,6 +78,7 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf auto& p = _parent->cast(); _part_type = p._part_type; + _hash_type = p._hash_type; std::map fragment_id_to_channel_index; for (int i = 0; i < p._dests.size(); ++i) { const auto& fragment_instance_id = p._dests[i].fragment_instance_id; @@ -132,9 +133,18 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf fmt::format("Crc32HashPartitioner({})", _partition_count)); } else if (_part_type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED) { _partition_count = channels.size(); - _partitioner = - std::make_unique>( - channels.size()); + if (_hash_type == THashType::SPARK_MURMUR32) { + _partitioner.reset( + new vectorized::Murmur32HashPartitioner( + channels.size())); + _profile->add_info_string("Partitioner", + fmt::format("Murmur32HashPartitioner({})", _partition_count)); + } else { + _partitioner.reset(new vectorized::Crc32HashPartitioner( + channels.size())); + _profile->add_info_string("Partitioner", + fmt::format("Crc32HashPartitioner({})", _partition_count)); + } RETURN_IF_ERROR(_partitioner->init(p._texprs)); RETURN_IF_ERROR(_partitioner->prepare(state, p._row_desc)); _profile->add_info_string("Partitioner", @@ -199,6 +209,8 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { RETURN_IF_ERROR(Base::open(state)); _writer.reset(new Writer()); auto& p = _parent->cast(); + _part_type = p._part_type; + _hash_type = p._hash_type; if (_part_type == TPartitionType::UNPARTITIONED || _part_type == TPartitionType::RANDOM || _part_type == TPartitionType::TABLE_SINK_RANDOM_PARTITIONED) { @@ -268,6 +280,7 @@ ExchangeSinkOperatorX::ExchangeSinkOperatorX( _texprs(sink.output_partition.partition_exprs), _row_desc(row_desc), _part_type(sink.output_partition.type), + _hash_type(sink.output_partition.hash_type), _dests(destinations), _dest_node_id(sink.dest_node_id), _transfer_large_data_by_brpc(config::transfer_large_data_by_brpc), @@ -289,6 +302,9 @@ ExchangeSinkOperatorX::ExchangeSinkOperatorX( sink.output_partition.type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED || sink.output_partition.type == TPartitionType::TABLE_SINK_HASH_PARTITIONED || sink.output_partition.type == TPartitionType::TABLE_SINK_RANDOM_PARTITIONED); + DCHECK(sink.output_partition.hash_type == THashType::CRC32 || + sink.output_partition.hash_type == THashType::XXHASH64 || + sink.output_partition.hash_type == THashType::SPARK_MURMUR32); _name = "ExchangeSinkOperatorX"; _pool = std::make_shared(); if (sink.__isset.output_tuple_id) { @@ -308,6 +324,28 @@ Status ExchangeSinkOperatorX::init(const TDataSink& tsink) { return Status::OK(); } +std::string ExchangeSinkOperatorX::debug_string(int indentation_level) const { + fmt::memory_buffer debug_string_buffer; + fmt::format_to(debug_string_buffer, "{}", Base::debug_string(indentation_level)); + + string dest_names; + for (const auto& dest : _dests) { + if (dest_names.empty()) { + dest_names += print_id(dest.fragment_instance_id); + } else { + dest_names += ", " + print_id(dest.fragment_instance_id); + } + } + + fmt::format_to(debug_string_buffer, + ", Info: (_num_recievers = {}, _dest_node_id = {}," + ", _partition_type = {}, _hash_type = {}," + " _destinations = [{}])", + _dests.size(), _dest_node_id, to_string(_part_type), to_string(_hash_type), + dest_names); + return fmt::to_string(debug_string_buffer); +} + Status ExchangeSinkOperatorX::open(RuntimeState* state) { RETURN_IF_ERROR(DataSinkOperatorX::open(state)); _state = state; @@ -385,7 +423,8 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block if (serialized) { auto cur_block = local_state._serializer.get_block()->to_block(); if (!cur_block.empty()) { - DCHECK(eos || local_state._serializer.is_local()) << debug_string(state, 0); + DCHECK(eos || local_state._serializer.is_local()) + << Base::debug_string(state, 0); RETURN_IF_ERROR(local_state._serializer.serialize_block( &cur_block, block_holder->get_block(), local_state._rpc_channels_num)); diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index 3d6eeb4b39e94f..9f55a3c7ecfc0a 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -174,6 +174,7 @@ class ExchangeSinkLocalState final : public PipelineXSinkLocalState<> { RuntimeProfile::Counter* _add_partition_request_timer = nullptr; TPartitionType::type _part_type; + THashType::type _hash_type; std::atomic _reach_limit = false; int _last_local_channel_idx = -1; @@ -184,6 +185,8 @@ class ExchangeSinkLocalState final : public PipelineXSinkLocalState<> { }; class ExchangeSinkOperatorX final : public DataSinkOperatorX { + using Base = DataSinkOperatorX; + public: ExchangeSinkOperatorX(RuntimeState* state, const RowDescriptor& row_desc, int operator_id, const TDataStreamSink& sink, @@ -191,6 +194,8 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorX& fragment_instance_ids); Status init(const TDataSink& tsink) override; + [[nodiscard]] std::string debug_string(int indentation_level) const override; + RuntimeState* state() { return _state; } Status open(RuntimeState* state) override; @@ -228,6 +233,7 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorX::debug_string(indentation_level)); - fmt::format_to(debug_string_buffer, ", Info: (_num_senders = {}, _is_merging = {})", - _num_senders, _is_merging); + fmt::format_to(debug_string_buffer, + ", Info: (_num_senders = {}, _is_merging = {}, _hash_type = {})", _num_senders, + _is_merging, to_string(_hash_type)); return fmt::to_string(debug_string_buffer); } @@ -106,6 +107,8 @@ ExchangeSourceOperatorX::ExchangeSourceOperatorX(ObjectPool* pool, const TPlanNo _partition_type(tnode.exchange_node.__isset.partition_type ? tnode.exchange_node.partition_type : TPartitionType::UNPARTITIONED), + _hash_type(tnode.exchange_node.__isset.hash_type ? tnode.exchange_node.hash_type + : THashType::CRC32), _input_row_desc(descs, tnode.exchange_node.input_row_tuples, std::vector(tnode.nullable_tuples.begin(), tnode.nullable_tuples.begin() + diff --git a/be/src/pipeline/exec/exchange_source_operator.h b/be/src/pipeline/exec/exchange_source_operator.h index ff9c5840033777..4b9cfe4b435418 100644 --- a/be/src/pipeline/exec/exchange_source_operator.h +++ b/be/src/pipeline/exec/exchange_source_operator.h @@ -91,7 +91,7 @@ class ExchangeSourceOperatorX final : public OperatorX { return _partition_type == TPartitionType::HASH_PARTITIONED ? DataDistribution(ExchangeType::HASH_SHUFFLE) : _partition_type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED - ? DataDistribution(ExchangeType::BUCKET_HASH_SHUFFLE) + ? DataDistribution(ExchangeType::BUCKET_HASH_SHUFFLE, _hash_type) : DataDistribution(ExchangeType::NOOP); } @@ -100,6 +100,7 @@ class ExchangeSourceOperatorX final : public OperatorX { const int _num_senders; const bool _is_merging; const TPartitionType::type _partition_type; + const THashType::type _hash_type; RowDescriptor _input_row_desc; // use in merge sort diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp index 852dccae71ca3b..b80c40345d8bae 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp @@ -403,6 +403,8 @@ PartitionedHashJoinSinkOperatorX::PartitionedHashJoinSinkOperatorX(ObjectPool* p descs), _join_distribution(tnode.hash_join_node.__isset.dist_type ? tnode.hash_join_node.dist_type : TJoinDistributionType::NONE), + _hash_type(tnode.hash_join_node.__isset.hash_type ? tnode.hash_join_node.hash_type + : THashType::CRC32), _distribution_partition_exprs(tnode.__isset.distribute_expr_lists ? tnode.distribute_expr_lists[1] : std::vector {}), diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h index e16e52dcaf9453..30b41d66c7530b 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h @@ -110,7 +110,7 @@ class PartitionedHashJoinSinkOperatorX return _join_distribution == TJoinDistributionType::BUCKET_SHUFFLE || _join_distribution == TJoinDistributionType::COLOCATE ? DataDistribution(ExchangeType::BUCKET_HASH_SHUFFLE, - _distribution_partition_exprs) + _distribution_partition_exprs, _hash_type) : DataDistribution(ExchangeType::HASH_SHUFFLE, _distribution_partition_exprs); } @@ -135,6 +135,7 @@ class PartitionedHashJoinSinkOperatorX Status _setup_internal_operator(RuntimeState* state); const TJoinDistributionType::type _join_distribution; + THashType::type _hash_type; std::vector _build_exprs; diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index 917a8283d5787b..9078731ee438ec 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -57,16 +57,16 @@ class SipHash; } \ } -#define DO_MURMUR_HASHES_FUNCTION_COLUMN_IMPL(SEED) \ - if (null_data == nullptr) { \ - for (size_t i = 0; i < s; i++) { \ - hashes[i] = HashUtil::murmur_hash3_32(&data[i], sizeof(T), SEED); \ - } \ - } else { \ - for (size_t i = 0; i < s; i++) { \ - if (null_data[i] == 0) \ - hashes[i] = HashUtil::murmur_hash3_32(&data[i], sizeof(T), SEED); \ - } \ +#define DO_MURMUR_HASHES_FUNCTION_COLUMN_IMPL() \ + if (null_data == nullptr) { \ + for (size_t i = 0; i < s; i++) { \ + hashes[i] = HashUtil::murmur_hash3_32(&data[i], sizeof(T), hashes[i]); \ + } \ + } else { \ + for (size_t i = 0; i < s; i++) { \ + if (null_data[i] == 0) \ + hashes[i] = HashUtil::murmur_hash3_32(&data[i], sizeof(T), hashes[i]); \ + } \ } namespace doris::vectorized { diff --git a/be/src/vec/columns/column_array.cpp b/be/src/vec/columns/column_array.cpp index 814aea9d6af83f..16a3fc256559b1 100644 --- a/be/src/vec/columns/column_array.cpp +++ b/be/src/vec/columns/column_array.cpp @@ -323,9 +323,6 @@ void ColumnArray::update_crcs_with_value(uint32_t* __restrict hash, PrimitiveTyp void ColumnArray::update_murmur_with_value(size_t start, size_t end, int32_t& hash, const uint8_t* __restrict null_data) const { auto& offsets_column = get_offsets(); - if (hash == 0) { - hash = HashUtil::SPARK_MURMUR_32_SEED; - } if (null_data) { for (size_t i = start; i < end; ++i) { if (null_data[i] == 0) { diff --git a/be/src/vec/columns/column_decimal.cpp b/be/src/vec/columns/column_decimal.cpp index 5b45b96afe628e..78c0d9162bbc2c 100644 --- a/be/src/vec/columns/column_decimal.cpp +++ b/be/src/vec/columns/column_decimal.cpp @@ -189,8 +189,7 @@ void ColumnDecimal::update_murmur_with_value(size_t start, size_t end, int32_ if (null_data == nullptr) { for (size_t i = start; i < end; i++) { if constexpr (!IsDecimalV2) { - hash = HashUtil::murmur_hash3_32(&data[i], sizeof(T), - HashUtil::SPARK_MURMUR_32_SEED); + hash = HashUtil::murmur_hash3_32(&data[i], sizeof(T), hash); } else { decimalv2_do_murmur(i, hash); } @@ -199,8 +198,7 @@ void ColumnDecimal::update_murmur_with_value(size_t start, size_t end, int32_ for (size_t i = start; i < end; i++) { if (null_data[i] == 0) { if constexpr (!IsDecimalV2) { - hash = HashUtil::murmur_hash3_32(&data[i], sizeof(T), - HashUtil::SPARK_MURMUR_32_SEED); + hash = HashUtil::murmur_hash3_32(&data[i], sizeof(T), hash); } else { decimalv2_do_murmur(i, hash); } @@ -217,7 +215,7 @@ void ColumnDecimal::update_murmurs_with_value(int32_t* __restrict hashes, Pri DCHECK(s == size()); if constexpr (!IsDecimalV2) { - DO_MURMUR_HASHES_FUNCTION_COLUMN_IMPL(HashUtil::SPARK_MURMUR_32_SEED) + DO_MURMUR_HASHES_FUNCTION_COLUMN_IMPL() } else { if (null_data == nullptr) { for (size_t i = 0; i < s; i++) { diff --git a/be/src/vec/columns/column_map.cpp b/be/src/vec/columns/column_map.cpp index 06017ed0f391a0..a095620ddfe04d 100644 --- a/be/src/vec/columns/column_map.cpp +++ b/be/src/vec/columns/column_map.cpp @@ -346,9 +346,6 @@ void ColumnMap::update_crc_with_value(size_t start, size_t end, uint32_t& hash, void ColumnMap::update_murmur_with_value(size_t start, size_t end, int32_t& hash, const uint8_t* __restrict null_data) const { auto& offsets = get_offsets(); - if (hash == 0) { - hash = HashUtil::SPARK_MURMUR_32_SEED; - } if (null_data) { for (size_t i = start; i < end; ++i) { if (null_data[i] == 0) { diff --git a/be/src/vec/columns/column_nullable.cpp b/be/src/vec/columns/column_nullable.cpp index 4f8de7f9cfe173..49854b30163fc5 100644 --- a/be/src/vec/columns/column_nullable.cpp +++ b/be/src/vec/columns/column_nullable.cpp @@ -94,9 +94,8 @@ void ColumnNullable::update_murmur_with_value(size_t start, size_t end, int32_t& nested_column->update_murmur_with_value(start, end, hash, nullptr); } else { const auto* __restrict real_null_data = - assert_cast(*null_map).get_data().data(); - hash = HashUtil::SPARK_MURMUR_32_SEED; - for (int i = start; i < end; ++i) { + assert_cast(get_null_map_column()).get_data().data(); + for (size_t i = start; i < end; ++i) { if (real_null_data[i] != 0) { hash = HashUtil::murmur_hash3_32_null(hash); } @@ -141,13 +140,13 @@ void ColumnNullable::update_murmurs_with_value(int32_t* __restrict hashes, auto s = rows; DCHECK(s == size()); const auto* __restrict real_null_data = - assert_cast(*null_map).get_data().data(); + assert_cast(get_null_map_column()).get_data().data(); if (!has_null()) { nested_column->update_murmurs_with_value(hashes, type, rows, offset, nullptr); } else { for (int i = 0; i < s; ++i) { if (real_null_data[i] != 0) { - hashes[i] = HashUtil::murmur_hash3_32_null(HashUtil::SPARK_MURMUR_32_SEED); + hashes[i] = HashUtil::murmur_hash3_32_null(hashes[i]); } } nested_column->update_murmurs_with_value(hashes, type, rows, offset, real_null_data); diff --git a/be/src/vec/columns/column_string.cpp b/be/src/vec/columns/column_string.cpp index e012fa51a0554b..fd6f09d88524ea 100644 --- a/be/src/vec/columns/column_string.cpp +++ b/be/src/vec/columns/column_string.cpp @@ -286,7 +286,8 @@ void ColumnStr::update_crcs_with_value(uint32_t* __restrict hashes, doris::Pr } } -void ColumnString::update_murmurs_with_value(int32_t* __restrict hashes, doris::PrimitiveType type, +template +void ColumnStr::update_murmurs_with_value(int32_t* __restrict hashes, doris::PrimitiveType type, int32_t rows, uint32_t offset, const uint8_t* __restrict null_data) const { auto s = rows; @@ -295,15 +296,13 @@ void ColumnString::update_murmurs_with_value(int32_t* __restrict hashes, doris:: if (null_data == nullptr) { for (size_t i = 0; i < s; i++) { auto data_ref = get_data_at(i); - hashes[i] = HashUtil::murmur_hash3_32(data_ref.data, data_ref.size, - HashUtil::SPARK_MURMUR_32_SEED); + hashes[i] = HashUtil::murmur_hash3_32(data_ref.data, data_ref.size, hashes[i]); } } else { for (size_t i = 0; i < s; i++) { if (null_data[i] == 0) { auto data_ref = get_data_at(i); - hashes[i] = HashUtil::murmur_hash3_32(data_ref.data, data_ref.size, - HashUtil::SPARK_MURMUR_32_SEED); + hashes[i] = HashUtil::murmur_hash3_32(data_ref.data, data_ref.size, hashes[i]); } } } diff --git a/be/src/vec/columns/column_string.h b/be/src/vec/columns/column_string.h index 044fcec952ce47..dd4ca1e69ed0b4 100644 --- a/be/src/vec/columns/column_string.h +++ b/be/src/vec/columns/column_string.h @@ -433,15 +433,13 @@ class ColumnStr final : public COWHelper> { for (size_t i = start; i < end; ++i) { if (null_data[i] == 0) { auto data_ref = get_data_at(i); - hash = HashUtil::murmur_hash3_32(data_ref.data, data_ref.size, - HashUtil::SPARK_MURMUR_32_SEED); + hash = HashUtil::murmur_hash3_32(data_ref.data, data_ref.size, hash); } } } else { for (size_t i = start; i < end; ++i) { auto data_ref = get_data_at(i); - hash = HashUtil::murmur_hash3_32(data_ref.data, data_ref.size, - HashUtil::SPARK_MURMUR_32_SEED); + hash = HashUtil::murmur_hash3_32(data_ref.data, data_ref.size, hash); } } } diff --git a/be/src/vec/columns/column_vector.cpp b/be/src/vec/columns/column_vector.cpp index 97a72265969a2a..1c75f26ec37ed9 100644 --- a/be/src/vec/columns/column_vector.cpp +++ b/be/src/vec/columns/column_vector.cpp @@ -220,14 +220,14 @@ void ColumnVector::update_murmurs_with_value(int32_t* __restrict hashes, Prim DCHECK(s == size()); if constexpr (!std::is_same_v) { - DO_MURMUR_HASHES_FUNCTION_COLUMN_IMPL(HashUtil::SPARK_MURMUR_32_SEED) + DO_MURMUR_HASHES_FUNCTION_COLUMN_IMPL() } else { if (type == TYPE_DATE || type == TYPE_DATETIME) { char buf[64]; auto date_convert_do_crc = [&](size_t i) { const VecDateTimeValue& date_val = (const VecDateTimeValue&)data[i]; auto len = date_val.to_buffer(buf); - hashes[i] = HashUtil::murmur_hash3_32(buf, len, HashUtil::SPARK_MURMUR_32_SEED); + hashes[i] = HashUtil::murmur_hash3_32(buf, len, hashes[i]); }; if (null_data == nullptr) { @@ -242,7 +242,7 @@ void ColumnVector::update_murmurs_with_value(int32_t* __restrict hashes, Prim } } } else { - DO_MURMUR_HASHES_FUNCTION_COLUMN_IMPL(HashUtil::SPARK_MURMUR_32_SEED) + DO_MURMUR_HASHES_FUNCTION_COLUMN_IMPL() } } } diff --git a/be/src/vec/columns/column_vector.h b/be/src/vec/columns/column_vector.h index 4d038958b74a67..7297f7a6efb05c 100644 --- a/be/src/vec/columns/column_vector.h +++ b/be/src/vec/columns/column_vector.h @@ -309,9 +309,6 @@ class ColumnVector final : public COWHelper> { void update_murmur_with_value(size_t start, size_t end, int32_t& hash, const uint8_t* __restrict null_data) const override { - if (hash == 0) { - hash = HashUtil::SPARK_MURMUR_32_SEED; - } if (null_data) { for (size_t i = start; i < end; i++) { if (null_data[i] == 0) { diff --git a/be/src/vec/runtime/partitioner.cpp b/be/src/vec/runtime/partitioner.cpp index 4a29e9689f722c..5f4f651c498706 100644 --- a/be/src/vec/runtime/partitioner.cpp +++ b/be/src/vec/runtime/partitioner.cpp @@ -17,18 +17,16 @@ #include "partitioner.h" -#include "common/cast_set.h" #include "pipeline/local_exchange/local_exchange_sink_operator.h" #include "runtime/thread_context.h" #include "vec/columns/column_const.h" #include "vec/sink/vdata_stream_sender.h" namespace doris::vectorized { -#include "common/compile_check_begin.h" template -Status Partitioner::do_partitioning(RuntimeState* state, Block* block, - MemTracker* mem_tracker) const { +Status Partitioner::do_partitioning(RuntimeState* state, + Block* block) const { int rows = block->rows(); if (rows > 0) { @@ -38,7 +36,7 @@ Status Partitioner::do_partitioning(RuntimeState* sta std::vector result(result_size); _hash_vals.resize(rows); - std::fill(_hash_vals.begin(), _hash_vals.end(), 0); + std::fill(_hash_vals.begin(), _hash_vals.end(), _get_default_seed()); auto* __restrict hashes = _hash_vals.data(); { RETURN_IF_ERROR(_get_partition_column_result(block, result)); } for (int j = 0; j < result_size; ++j) { @@ -61,7 +59,7 @@ Status Partitioner::do_partitioning(RuntimeState* sta template void Crc32HashPartitioner::_do_hash(const ColumnPtr& column, uint32_t* __restrict result, int idx) const { - column->update_crcs_with_value(result, _partition_expr_ctxs[idx]->root()->type().type, + column->update_crcs_with_value(result, Base::_partition_expr_ctxs[idx]->root()->type().type, cast_set(column->size())); } @@ -75,12 +73,13 @@ void Murmur32HashPartitioner::_do_hash(const ColumnPtr& column, template Status Crc32HashPartitioner::clone(RuntimeState* state, std::unique_ptr& partitioner) { - auto* new_partitioner = new Crc32HashPartitioner(cast_set(_partition_count)); + auto* new_partitioner = + new Crc32HashPartitioner(cast_set(Base::_partition_count)); partitioner.reset(new_partitioner); - new_partitioner->_partition_expr_ctxs.resize(_partition_expr_ctxs.size()); - for (size_t i = 0; i < _partition_expr_ctxs.size(); i++) { - RETURN_IF_ERROR( - _partition_expr_ctxs[i]->clone(state, new_partitioner->_partition_expr_ctxs[i])); + new_partitioner->_partition_expr_ctxs.resize(Base::_partition_expr_ctxs.size()); + for (size_t i = 0; i < Base::_partition_expr_ctxs.size(); i++) { + RETURN_IF_ERROR(Base::_partition_expr_ctxs[i]->clone( + state, new_partitioner->_partition_expr_ctxs[i])); } return Status::OK(); } @@ -88,7 +87,8 @@ Status Crc32HashPartitioner::clone(RuntimeState* state, template Status Murmur32HashPartitioner::clone(RuntimeState* state, std::unique_ptr& partitioner) { - auto* new_partitioner = new Murmur32HashPartitioner(Base::_partition_count); + auto* new_partitioner = + new Murmur32HashPartitioner(cast_set(Base::_partition_count)); partitioner.reset(new_partitioner); new_partitioner->_partition_expr_ctxs.resize(Base::_partition_expr_ctxs.size()); for (size_t i = 0; i < Base::_partition_expr_ctxs.size(); i++) { @@ -98,6 +98,11 @@ Status Murmur32HashPartitioner::clone(RuntimeState* state, return Status::OK(); } +template +int32_t Murmur32HashPartitioner::_get_default_seed() const { + return static_cast(HashUtil::SPARK_MURMUR_32_SEED); +} + template class Crc32HashPartitioner; template class Crc32HashPartitioner; template class Murmur32HashPartitioner; diff --git a/be/src/vec/runtime/partitioner.h b/be/src/vec/runtime/partitioner.h index 6556b91d6ef4a6..c653e0f64aade2 100644 --- a/be/src/vec/runtime/partitioner.h +++ b/be/src/vec/runtime/partitioner.h @@ -36,7 +36,7 @@ struct ChannelField { class PartitionerBase { public: - PartitionerBase(size_t partition_count) : _partition_count(partition_count) {} + PartitionerBase(const size_t partition_count) : _partition_count(partition_count) {} virtual ~PartitionerBase() = default; virtual Status init(const std::vector& texprs) = 0; @@ -59,11 +59,11 @@ class PartitionerBase { const size_t _partition_count; }; -template -class Crc32HashPartitioner : public PartitionerBase { +template +class Partitioner : public PartitionerBase { public: - Crc32HashPartitioner(int partition_count) : PartitionerBase(partition_count) {} - ~Crc32HashPartitioner() override = default; + Partitioner(const size_t partition_count) : PartitionerBase(partition_count) {} + ~Partitioner() override = default; Status init(const std::vector& texprs) override { return VExpr::create_expr_trees(texprs, _partition_expr_ctxs); @@ -79,9 +79,9 @@ class Crc32HashPartitioner : public PartitionerBase { Status do_partitioning(RuntimeState* state, Block* block) const override; - ChannelField get_channel_ids() const override { return {_hash_vals.data(), sizeof(uint32_t)}; } - - Status clone(RuntimeState* state, std::unique_ptr& partitioner) override; + ChannelField get_channel_ids() const override { + return {_hash_vals.data(), sizeof(HashValueType)}; + } protected: Status _get_partition_column_result(Block* block, std::vector& result) const { @@ -92,10 +92,12 @@ class Crc32HashPartitioner : public PartitionerBase { return Status::OK(); } - void _do_hash(const ColumnPtr& column, uint32_t* __restrict result, int idx) const; + virtual void _do_hash(const ColumnPtr& column, HashValueType* __restrict result, + int idx) const = 0; + virtual HashValueType _get_default_seed() const { return static_cast(0); } VExprContextSPtrs _partition_expr_ctxs; - mutable std::vector _hash_vals; + mutable std::vector _hash_vals; }; struct ShuffleChannelIds { @@ -112,11 +114,32 @@ struct SpillPartitionChannelIds { } }; +struct ShufflePModChannelIds { + template + HashValueType operator()(HashValueType l, int32_t r) { + return (l % r + r) % r; + } +}; + +template +class Crc32HashPartitioner final : public Partitioner { +public: + using Base = Partitioner; + Crc32HashPartitioner(size_t partition_count) + : Partitioner(partition_count) {} + ~Crc32HashPartitioner() override = default; + + Status clone(RuntimeState* state, std::unique_ptr& partitioner) override; + +private: + void _do_hash(const ColumnPtr& column, uint32_t* __restrict result, int idx) const override; +}; + template class Murmur32HashPartitioner final : public Partitioner { public: using Base = Partitioner; - Murmur32HashPartitioner(int partition_count) + Murmur32HashPartitioner(size_t partition_count) : Partitioner(partition_count) {} ~Murmur32HashPartitioner() override = default; @@ -124,10 +147,8 @@ class Murmur32HashPartitioner final : public Partitioner { private: void _do_hash(const ColumnPtr& column, int32_t* __restrict result, int idx) const override; + int32_t _get_default_seed() const override; }; -} // namespace vectorized -} // namespace doris - #include "common/compile_check_end.h" } // namespace doris::vectorized diff --git a/be/src/vec/sink/vdata_stream_sender.h b/be/src/vec/sink/vdata_stream_sender.h index 8c3aab6622b693..0ff1f252d5441f 100644 --- a/be/src/vec/sink/vdata_stream_sender.h +++ b/be/src/vec/sink/vdata_stream_sender.h @@ -100,13 +100,6 @@ class BlockSerializer { const int _batch_size; }; -struct ShufflePModChannelIds { - template - HashValueType operator()(HashValueType l, int32_t r) { - return (l % r + r) % r; - } -}; - class Channel { public: friend class pipeline::ExchangeSinkBuffer; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/FederationBackendPolicy.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/FederationBackendPolicy.java index 639a2bd715f7ed..bc7a95d0eacca0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/FederationBackendPolicy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/FederationBackendPolicy.java @@ -23,9 +23,11 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.Config; import org.apache.doris.common.IndexedPriorityQueue; +import org.apache.doris.common.Pair; import org.apache.doris.common.ResettableRandomizedIterator; import org.apache.doris.common.UserException; import org.apache.doris.common.util.ConsistentHash; +import org.apache.doris.datasource.hive.HiveBucketUtil; import org.apache.doris.mysql.privilege.UserProperty; import org.apache.doris.qe.ConnectContext; import org.apache.doris.resource.Tag; @@ -81,6 +83,7 @@ public Map getAssignedWeightPerBackend() { private Map assignedWeightPerBackend = Maps.newHashMap(); protected ConsistentHash consistentHash; + protected ConsistentHash consistentBucketHash; private int nextBe = 0; private boolean initialized = false; @@ -200,6 +203,8 @@ public void init(BeSelectionPolicy policy) throws UserException { backendMap.putAll(backends.stream().collect(Collectors.groupingBy(Backend::getHost))); try { consistentHash = consistentHashCache.get(new HashCacheKey(backends)); + consistentBucketHash = new ConsistentHash<>(Hashing.murmur3_128(), new BucketHash(), + new BackendHash(), backends, Config.split_assigner_virtual_node_number); } catch (ExecutionException e) { throw new UserException("failed to get consistent hash", e); } @@ -216,6 +221,21 @@ public void setEnableSplitsRedistribution(boolean enableSplitsRedistribution) { this.enableSplitsRedistribution = enableSplitsRedistribution; } + public Multimap, Split> computeBucketAwareScanRangeAssignmentWith(List splits) + throws UserException { + ListMultimap, Split> assignment = ArrayListMultimap.create(); + int bucketNum = 0; + for (Split split : splits) { + FileSplit fileSplit = (FileSplit) split; + bucketNum = HiveBucketUtil.getBucketNumberFromPath(fileSplit.getPath().getPath().getName()).getAsInt(); + + List candidateNodes = consistentBucketHash.getNode(bucketNum, 1); + assignment.put(Pair.of(candidateNodes.get(0), bucketNum), split); + } + + return assignment; + } + /** * Assign splits to each backend. Ensure that each backend receives a similar amount of data. * In order to make sure backends utilize the os page cache as much as possible, and all backends read splits diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java index caebca70cf57c0..f9e39e3ef64ed4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java @@ -30,13 +30,13 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.NotImplementedException; +import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.BrokerUtil; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.hive.AcidInfo; import org.apache.doris.datasource.hive.AcidInfo.DeleteDeltaInfo; import org.apache.doris.datasource.hive.HMSExternalTable; -import org.apache.doris.datasource.hive.HiveBucketUtil; import org.apache.doris.datasource.hive.source.HiveScanNode; import org.apache.doris.datasource.hive.source.HiveSplit; import org.apache.doris.planner.DataPartition; @@ -68,6 +68,7 @@ import org.apache.doris.thrift.TTransactionalHiveDeleteDeltaDesc; import org.apache.doris.thrift.TTransactionalHiveDesc; +import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Lists; @@ -91,11 +92,11 @@ public abstract class FileQueryScanNode extends FileScanNode { private static final Logger LOG = LogManager.getLogger(FileQueryScanNode.class); + public ArrayListMultimap bucketSeq2locations = ArrayListMultimap.create(); + protected Map destSlotDescByName; protected TFileScanRangeParams params; - public ArrayListMultimap bucketSeq2locations = ArrayListMultimap.create(); - @Getter protected TableSample tableSample; @@ -281,6 +282,13 @@ protected Optional getSerializedTable() { @Override public void createScanRangeLocations() throws UserException { + if (!scanRangeLocations.isEmpty()) { + /* Note: createScanRangeLocations invoked twice thru finalizeForNereids() + * and produced duplicate records. + */ + return; + } + long start = System.currentTimeMillis(); if (ConnectContext.get().getExecutor() != null) { ConnectContext.get().getExecutor().getSummaryProfile().setGetSplitsStartTime(); @@ -326,13 +334,20 @@ public void createScanRangeLocations() throws UserException { params.setProperties(locationProperties); } + boolean isSparkBucketedHiveTable = false; + TableIf targetTable = getTargetTable(); + if (targetTable instanceof HMSExternalTable) { + isSparkBucketedHiveTable = ((HMSExternalTable) targetTable).isSparkBucketedTable(); + } + int numBackends = backendPolicy.numBackends(); List pathPartitionKeys = getPathPartitionKeys(); if (isBatchMode()) { // File splits are generated lazily, and fetched by backends while scanning. // Only provide the unique ID of split source to backend. splitAssignment = new SplitAssignment( - backendPolicy, this, this::splitToScanRange, locationProperties, pathPartitionKeys); + backendPolicy, this, this::splitToScanRange, locationProperties, pathPartitionKeys, + isSparkBucketedHiveTable); splitAssignment.init(); if (ConnectContext.get().getExecutor() != null) { ConnectContext.get().getExecutor().getSummaryProfile().setGetSplitsFinishTime(); @@ -360,76 +375,15 @@ public void createScanRangeLocations() throws UserException { tSource.setSplitSourceId(splitSource.getUniqueId()); tSource.setNumSplits(numSplitsPerBE); curLocations.getScanRange().getExtScanRange().getFileScanRange().setSplitSource(tSource); -/*======= - // If fileSplit has partition values, use the values collected from hive partitions. - // Otherwise, use the values in file path. - boolean isACID = false; - if (fileSplit instanceof HiveSplit) { - HiveSplit hiveSplit = (HiveSplit) fileSplit; - isACID = hiveSplit.isACID(); - } - List partitionValuesFromPath = fileSplit.getPartitionValues() == null - ? BrokerUtil.parseColumnsFromPath(fileSplit.getPath().toString(), pathPartitionKeys, - false, isACID) : fileSplit.getPartitionValues(); - - boolean isSparkBucketedHiveTable = false; - int bucketNum = 0; - TableIf targetTable = getTargetTable(); - if (targetTable instanceof HMSExternalTable) { - isSparkBucketedHiveTable = ((HMSExternalTable) targetTable).isSparkBucketedTable(); - if (isSparkBucketedHiveTable) { - bucketNum = HiveBucketUtil.getBucketNumberFromPath(fileSplit.getPath().getName()).getAsInt(); - } - } - TFileRangeDesc rangeDesc = createFileRangeDesc(fileSplit, partitionValuesFromPath, pathPartitionKeys, - locationType); - TFileCompressType fileCompressType = getFileCompressType(fileSplit); - rangeDesc.setCompressType(fileCompressType); - if (isACID) { - HiveSplit hiveSplit = (HiveSplit) fileSplit; - hiveSplit.setTableFormatType(TableFormatType.TRANSACTIONAL_HIVE); - TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); - tableFormatFileDesc.setTableFormatType(hiveSplit.getTableFormatType().value()); - AcidInfo acidInfo = (AcidInfo) hiveSplit.getInfo(); - TTransactionalHiveDesc transactionalHiveDesc = new TTransactionalHiveDesc(); - transactionalHiveDesc.setPartition(acidInfo.getPartitionLocation()); - List deleteDeltaDescs = new ArrayList<>(); - for (DeleteDeltaInfo deleteDeltaInfo : acidInfo.getDeleteDeltas()) { - TTransactionalHiveDeleteDeltaDesc deleteDeltaDesc = new TTransactionalHiveDeleteDeltaDesc(); - deleteDeltaDesc.setDirectoryLocation(deleteDeltaInfo.getDirectoryLocation()); - deleteDeltaDesc.setFileNames(deleteDeltaInfo.getFileNames()); - deleteDeltaDescs.add(deleteDeltaDesc); - } - transactionalHiveDesc.setDeleteDeltas(deleteDeltaDescs); - tableFormatFileDesc.setTransactionalHiveParams(transactionalHiveDesc); - rangeDesc.setTableFormatParams(tableFormatFileDesc); - } - setScanParams(rangeDesc, fileSplit); - - curLocations.getScanRange().getExtScanRange().getFileScanRange().addToRanges(rangeDesc); ->>>>>>> a5ce2395a2 ([feature](datalake) Add BucketShuffleJoin support for Hive table data generated by Spark. (27783)) -*/ TScanRangeLocation location = new TScanRangeLocation(); location.setBackendId(backend.getId()); location.setServer(new TNetworkAddress(backend.getHost(), backend.getBePort())); curLocations.addToLocations(location); - // So there's only one scan range for each backend. + // So there's only one scan range for each backend. // Each backend only starts up one ScanNode instance. // However, even one ScanNode instance can provide maximum scanning concurrency. -/*======= - if (LOG.isDebugEnabled()) { - LOG.debug("assign to backend {} with table split: {} ({}, {}), location: {}", - curLocations.getLocations().get(0).getBackendId(), fileSplit.getPath(), - fileSplit.getStart(), fileSplit.getLength(), - Joiner.on("|").join(fileSplit.getHosts())); - } - if (isSparkBucketedHiveTable) { - bucketSeq2locations.put(bucketNum, curLocations); - } ->>>>>>> a5ce2395a2 ([feature](datalake) Add BucketShuffleJoin support for Hive table data generated by Spark. (27783)) -*/ scanRangeLocations.add(curLocations); setLocationPropertiesIfNecessary(backend, locationType, locationProperties); scanBackendIds.add(backend.getId()); @@ -443,14 +397,33 @@ >>>>>>> a5ce2395a2 ([feature](datalake) Add BucketShuffleJoin support for Hive t if (inputSplits.isEmpty() && !isFileStreamType()) { return; } - Multimap assignment = backendPolicy.computeScanRangeAssignment(inputSplits); - for (Backend backend : assignment.keySet()) { - Collection splits = assignment.get(backend); - for (Split split : splits) { - scanRangeLocations.add(splitToScanRange(backend, locationProperties, split, pathPartitionKeys)); - totalFileSize += split.getLength(); + + if (isSparkBucketedHiveTable) { + Multimap, Split> assignment; + + assignment = backendPolicy.computeBucketAwareScanRangeAssignmentWith(inputSplits); + for (Pair backend : assignment.keySet()) { + Collection splits = assignment.get(backend); + for (Split split : splits) { + scanRangeLocations.add(splitToScanRange(backend.first, backend.second, locationProperties, + split, pathPartitionKeys)); + totalFileSize += split.getLength(); + } + scanBackendIds.add(backend.first.getId()); + } + } else { + Multimap assignment; + + assignment = backendPolicy.computeScanRangeAssignment(inputSplits); + for (Backend backend : assignment.keySet()) { + Collection splits = assignment.get(backend); + for (Split split : splits) { + scanRangeLocations.add(splitToScanRange(backend, 0, locationProperties, split, + pathPartitionKeys)); + totalFileSize += split.getLength(); + } + scanBackendIds.add(backend.getId()); } - scanBackendIds.add(backend.getId()); } } @@ -467,6 +440,7 @@ >>>>>>> a5ce2395a2 ([feature](datalake) Add BucketShuffleJoin support for Hive t private TScanRangeLocations splitToScanRange( Backend backend, + Integer bucketNum, Map locationProperties, Split split, List pathPartitionKeys) throws UserException { @@ -483,6 +457,9 @@ private TScanRangeLocations splitToScanRange( ? BrokerUtil.parseColumnsFromPath(fileSplit.getPathString(), pathPartitionKeys, false, isACID) : fileSplit.getPartitionValues(); + boolean isSparkBucketedHiveTable = false; + TableIf targetTable = getTargetTable(); + TFileRangeDesc rangeDesc = createFileRangeDesc(fileSplit, partitionValuesFromPath, pathPartitionKeys); TFileCompressType fileCompressType = getFileCompressType(fileSplit); rangeDesc.setCompressType(fileCompressType); @@ -521,6 +498,23 @@ private TScanRangeLocations splitToScanRange( location.setBackendId(backend.getId()); location.setServer(new TNetworkAddress(backend.getHost(), backend.getBePort())); curLocations.addToLocations(location); + + if (targetTable instanceof HMSExternalTable) { + isSparkBucketedHiveTable = ((HMSExternalTable) targetTable).isSparkBucketedTable(); + if (isSparkBucketedHiveTable) { + if (!bucketSeq2locations.containsKey(bucketNum)) { + bucketSeq2locations.put(bucketNum, curLocations); + } + } + } + + if (LOG.isDebugEnabled()) { + LOG.debug("assign to backend {} with table split: {} ({}, {}), location: {}, bucketNum: {}", + curLocations.getLocations().get(0).getBackendId(), fileSplit.getPath(), + fileSplit.getStart(), fileSplit.getLength(), + Joiner.on("|").join(fileSplit.getHosts()), bucketNum); + } + return curLocations; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitAssignment.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitAssignment.java index a26abc7fc5e037..0e34fae4043e8e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitAssignment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitAssignment.java @@ -17,11 +17,13 @@ package org.apache.doris.datasource; +import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.spi.Split; import org.apache.doris.system.Backend; import org.apache.doris.thrift.TScanRangeLocations; +import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; import java.util.ArrayList; @@ -49,6 +51,7 @@ public class SplitAssignment { private final Map locationProperties; private final List pathPartitionKeys; private final Object assignLock = new Object(); + private final boolean useBucketAssignment; private Split sampleSplit = null; private final AtomicBoolean isStop = new AtomicBoolean(false); private final AtomicBoolean scheduleFinished = new AtomicBoolean(false); @@ -60,12 +63,14 @@ public SplitAssignment( SplitGenerator splitGenerator, SplitToScanRange splitToScanRange, Map locationProperties, - List pathPartitionKeys) { + List pathPartitionKeys, + boolean useBucketedAssignment) { this.backendPolicy = backendPolicy; this.splitGenerator = splitGenerator; this.splitToScanRange = splitToScanRange; this.locationProperties = locationProperties; this.pathPartitionKeys = pathPartitionKeys; + this.useBucketAssignment = useBucketedAssignment; } public void init() throws UserException { @@ -88,14 +93,15 @@ private boolean waitFirstSplit() { return !scheduleFinished.get() && !isStop.get() && exception == null; } - private void appendBatch(Multimap batch) throws UserException { - for (Backend backend : batch.keySet()) { + private void appendBatch(Multimap, Split> batch) throws UserException { + for (Pair backend : batch.keySet()) { Collection splits = batch.get(backend); List locations = new ArrayList<>(splits.size()); for (Split split : splits) { - locations.add(splitToScanRange.getScanRange(backend, locationProperties, split, pathPartitionKeys)); + locations.add(splitToScanRange.getScanRange(backend.first, backend.second, locationProperties, + split, pathPartitionKeys)); } - if (!assignment.computeIfAbsent(backend, be -> new LinkedBlockingQueue<>()).offer(locations)) { + if (!assignment.computeIfAbsent(backend.first, be -> new LinkedBlockingQueue<>()).offer(locations)) { throw new UserException("Failed to offer batch split"); } } @@ -117,14 +123,20 @@ public void addToQueue(List splits) { if (splits.isEmpty()) { return; } - Multimap batch = null; + Multimap, Split> batch = ArrayListMultimap.create(); synchronized (assignLock) { if (sampleSplit == null) { sampleSplit = splits.get(0); assignLock.notify(); } try { - batch = backendPolicy.computeScanRangeAssignment(splits); + if (useBucketAssignment) { + batch = backendPolicy.computeBucketAwareScanRangeAssignmentWith(splits); + } else { + Multimap, Split> finalBatch = batch; + backendPolicy.computeScanRangeAssignment(splits).entries() + .forEach(e -> finalBatch.put(Pair.of(e.getKey(), 0), e.getValue())); + } } catch (UserException e) { exception = e; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitToScanRange.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitToScanRange.java index 0e890252857583..ea58b6d8d0bb0a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitToScanRange.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitToScanRange.java @@ -28,6 +28,7 @@ public interface SplitToScanRange { TScanRangeLocations getScanRange( Backend backend, + Integer bucketNum, Map locationProperties, Split split, List pathPartitionKeys) throws UserException; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index aa49d7d601ed8f..bfd157aba08911 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -64,6 +64,7 @@ import org.apache.doris.thrift.TTableType; import com.google.common.collect.BiMap; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -81,14 +82,13 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.StringColumnStatsData; import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.io.IOException; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.time.LocalDate; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -577,7 +577,7 @@ public Optional initSchema() { private Optional getIcebergSchema() { List columns = IcebergUtils.getSchema(catalog, dbName, name); - List partitionColumns = initPartitionColumns(columns); + partitionColumns = initPartitionColumns(columns); initBucketingColumns(columns); return Optional.of(new HMSSchemaCacheValue(columns, partitionColumns)); } @@ -654,7 +654,7 @@ private Optional getHudiSchema() { true, null, true, null, "", true, null, -1, null)); colTypes.add(HudiUtils.convertAvroToHiveType(hudiField.schema())); } - List partitionColumns = initPartitionColumns(tmpSchema); + partitionColumns = initPartitionColumns(tmpSchema); HudiSchemaCacheValue hudiSchemaCacheValue = new HudiSchemaCacheValue(tmpSchema, partitionColumns); hudiSchemaCacheValue.setColTypes(colTypes); return Optional.of(hudiSchemaCacheValue); @@ -672,7 +672,7 @@ private Optional getHiveSchema() { HiveMetaStoreClientHelper.hiveTypeToDorisType(field.getType()), true, null, true, defaultValue, field.getComment(), true, -1)); } - List partitionColumns = initPartitionColumns(columns); + partitionColumns = initPartitionColumns(columns); return Optional.of(new HMSSchemaCacheValue(columns, partitionColumns)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveBucketUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveBucketUtil.java index fc0bed8d5e1c25..41d04507a1deba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveBucketUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveBucketUtil.java @@ -103,7 +103,7 @@ private static PrimitiveTypeInfo convertToHiveColType(PrimitiveType dorisType) t private static final Iterable BUCKET_PATTERNS = ImmutableList.of( // spark/parquet pattern - // format: f"part-[paritionId]-[tid]-[txnId]-[jobId]-[taskAttemptId]-[fileCount].c000.snappy.parquet" + // format: f"part-[paritionId]-[tid]-[txnId]-[jobId]-[taskAttemptId]_[fileCount].c000.snappy.parquet" Pattern.compile("part-\\d{5}-\\w{8}-\\w{4}-\\w{4}-\\w{4}-\\w{12}_(\\d{5})(?:[-_.].*)?"), // legacy Presto naming pattern (current version matches Hive) Pattern.compile("\\d{8}_\\d{6}_\\d{5}_[a-z0-9]{5}_bucket-(\\d+)(?:[-_.].*)?"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java index e1d65125908ef7..9c40bc3731e524 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/source/HiveScanNode.java @@ -507,7 +507,7 @@ protected TFileCompressType getFileCompressType(FileSplit fileSplit) throws User @Override public DataPartition constructInputPartitionByDistributionInfo() { - if (hmsTable.isSparkBucketedTable()) { + if (hmsTable.isSparkBucketedTable() && ConnectContext.get().getSessionVariable().isEnableSparkShuffle()) { DistributionInfo distributionInfo = hmsTable.getDefaultDistributionInfo(); if (!(distributionInfo instanceof HashDistributionInfo)) { return DataPartition.RANDOM; @@ -530,7 +530,7 @@ public HMSExternalTable getHiveTable() { @Override public THashType getHashType() { - if (hmsTable.isSparkBucketedTable() + if (hmsTable.isSparkBucketedTable() && ConnectContext.get().getSessionVariable().isEnableSparkShuffle() && hmsTable.getDefaultDistributionInfo() instanceof HashDistributionInfo) { return THashType.SPARK_MURMUR32; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index b44ea8e5631638..6ba9a4bcd2bcd1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -348,6 +348,7 @@ public PlanFragment visitPhysicalDistribute(PhysicalDistribute d } DataPartition dataPartition = toDataPartition(distribute.getDistributionSpec(), validOutputIds, context); exchangeNode.setPartitionType(dataPartition.getType()); + exchangeNode.setHashType(dataPartition.getHashType()); exchangeNode.setChildrenDistributeExprLists(distributeExprLists); PlanFragment parentFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, dataPartition); if (distribute.getDistributionSpec() instanceof DistributionSpecGather) { @@ -558,7 +559,6 @@ public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTransla SessionVariable sv = ConnectContext.get().getSessionVariable(); // TODO(cmy): determine the needCheckColumnPriv param FileQueryScanNode scanNode; - DataPartition dataPartition = DataPartition.RANDOM; if (table instanceof HMSExternalTable) { switch (((HMSExternalTable) table).getDlaType()) { case ICEBERG: @@ -590,8 +590,8 @@ public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTransla } else { throw new RuntimeException("do not support table type " + table.getType()); } - if (fileScan.getTableSnapshot().isPresent() && scanNode instanceof FileQueryScanNode) { - ((FileQueryScanNode) scanNode).setQueryTableSnapshot(fileScan.getTableSnapshot().get()); + if (fileScan.getTableSnapshot().isPresent()) { + scanNode.setQueryTableSnapshot(fileScan.getTableSnapshot().get()); } return getPlanFragmentForPhysicalFileScan(fileScan, context, scanNode, table, tupleDescriptor); } @@ -690,9 +690,9 @@ private PlanFragment getPlanFragmentForPhysicalFileScan(PhysicalFileScan fileSca if (fileScan.getDistributionSpec() instanceof DistributionSpecHash) { DistributionSpecHash distributionSpecHash = (DistributionSpecHash) fileScan.getDistributionSpec(); List partitionExprs = distributionSpecHash.getOrderedShuffledColumns().stream() - .map(context::findSlotRef).collect(Collectors.toList()); + .map(context::findSlotRef).collect(Collectors.toList()); dataPartition = new DataPartition(TPartitionType.HASH_PARTITIONED, - partitionExprs, scanNode.getHashType()); + partitionExprs, ((FileQueryScanNode) scanNode).getHashType()); } // Create PlanFragment PlanFragment planFragment = createPlanFragment(scanNode, dataPartition, fileScan); @@ -1428,6 +1428,8 @@ public PlanFragment visitPhysicalHashJoin( hashJoinNode.setDistributionMode(DistributionMode.BROADCAST); } else if (JoinUtils.shouldBucketShuffleJoin(physicalHashJoin)) { hashJoinNode.setDistributionMode(DistributionMode.BUCKET_SHUFFLE); + hashJoinNode.setHashType(((DistributionSpecHash) physicalHashJoin.left() + .getPhysicalProperties().getDistributionSpec()).getShuffleFunction()); } else { hashJoinNode.setDistributionMode(DistributionMode.PARTITIONED); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java index 7b9d696cabc5e9..a68d44cc3118aa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java @@ -503,6 +503,8 @@ private PhysicalProperties computeShuffleJoinOutputProperties( ShuffleType outputShuffleType = shuffleSide == ShuffleSide.LEFT ? rightHashSpec.getShuffleType() : leftHashSpec.getShuffleType(); + DistributionSpecHash.StorageBucketHashType outputShuffleFunction = shuffleSide == ShuffleSide.LEFT + ? rightHashSpec.getShuffleFunction() : leftHashSpec.getShuffleFunction(); switch (hashJoin.getJoinType()) { case INNER_JOIN: @@ -522,7 +524,7 @@ private PhysicalProperties computeShuffleJoinOutputProperties( case LEFT_OUTER_JOIN: if (shuffleSide == ShuffleSide.LEFT) { return new PhysicalProperties( - leftHashSpec.withShuffleTypeAndForbidColocateJoin(outputShuffleType) + leftHashSpec.withShuffleTypeAndForbidColocateJoin(outputShuffleType, outputShuffleFunction) ); } else { return new PhysicalProperties(leftHashSpec); @@ -536,7 +538,7 @@ private PhysicalProperties computeShuffleJoinOutputProperties( // retain left shuffle type, since coordinator use left most node to schedule fragment // forbid colocate join, since right table already shuffle return new PhysicalProperties(rightHashSpec.withShuffleTypeAndForbidColocateJoin( - leftHashSpec.getShuffleType())); + leftHashSpec.getShuffleType(), leftHashSpec.getShuffleFunction())); } case FULL_OUTER_JOIN: return PhysicalProperties.createAnyFromHash(leftHashSpec, rightHashSpec); @@ -582,7 +584,7 @@ private PhysicalProperties legacyComputeShuffleJoinOutputProperties( // retain left shuffle type, since coordinator use left most node to schedule fragment // forbid colocate join, since right table already shuffle return new PhysicalProperties(rightHashSpec.withShuffleTypeAndForbidColocateJoin( - leftHashSpec.getShuffleType())); + leftHashSpec.getShuffleType(), leftHashSpec.getShuffleFunction())); } case FULL_OUTER_JOIN: return PhysicalProperties.createAnyFromHash(leftHashSpec); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java index d786215692cbb2..5cca6464afff65 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java @@ -679,7 +679,7 @@ private PhysicalProperties calAnotherSideRequired(ShuffleType shuffleType, notNeedShuffleSideRequired, needShuffleSideRequired); return new PhysicalProperties(new DistributionSpecHash(shuffleSideIds, shuffleType, needShuffleSideOutput.getTableId(), needShuffleSideOutput.getSelectedIndexId(), - needShuffleSideOutput.getPartitionIds(), notShuffleSideOutput.getShuffleFunction())); + needShuffleSideOutput.getPartitionIds(), notNeedShuffleSideOutput.getShuffleFunction())); } private void updateChildEnforceAndCost(int index, PhysicalProperties targetProperties) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java index 3bd1ab1ae52964..75e5682d924550 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java @@ -346,6 +346,9 @@ public enum StorageBucketHashType { // SPARK_MURMUR32 is the hash function for Spark bucketed hive table's storage and computation STORAGE_BUCKET_SPARK_MURMUR32; + /** + * convert to thrift + */ public THashType toThrift() { switch (this) { case STORAGE_BUCKET_CRC32: @@ -353,8 +356,25 @@ public THashType toThrift() { case STORAGE_BUCKET_SPARK_MURMUR32: return THashType.SPARK_MURMUR32; case STORAGE_BUCKET_XXHASH64: - default: return THashType.XXHASH64; + default: + return THashType.CRC32; + } + } + + /** + * convert from thrift + */ + public static StorageBucketHashType fromThrift(THashType hashType) { + switch (hashType) { + case CRC32: + return STORAGE_BUCKET_CRC32; + case SPARK_MURMUR32: + return STORAGE_BUCKET_SPARK_MURMUR32; + case XXHASH64: + return STORAGE_BUCKET_XXHASH64; + default: + return STORAGE_BUCKET_CRC32; } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java index 5162949ed796d1..54ffd42457fc80 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java @@ -28,12 +28,13 @@ import org.apache.doris.nereids.properties.DistributionSpecStorageAny; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.trees.plans.logical.LogicalHudiScan; import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalHudiScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalFileScan; +import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; @@ -68,8 +69,18 @@ private DistributionSpec convertDistribution(LogicalFileScan fileScan) { } HMSExternalTable hmsExternalTable = (HMSExternalTable) table; + if (hmsExternalTable.getDlaType() != HMSExternalTable.DLAType.HIVE + && !hmsExternalTable.isSparkBucketedTable()) { + return DistributionSpecStorageAny.INSTANCE; + } + + boolean isSelectUnpartitioned = !hmsExternalTable.isPartitionedTable() + || hmsExternalTable.getPartitionNames().size() == 1 + || fileScan.getSelectedPartitions().selectedPartitions.size() == 1; + DistributionInfo distributionInfo = hmsExternalTable.getDefaultDistributionInfo(); - if (distributionInfo instanceof HashDistributionInfo) { + if (distributionInfo instanceof HashDistributionInfo && isSelectUnpartitioned + && ConnectContext.get().getSessionVariable().isEnableSparkShuffle()) { HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo; List output = fileScan.getOutput(); List hashColumns = Lists.newArrayList(); @@ -80,12 +91,10 @@ private DistributionSpec convertDistribution(LogicalFileScan fileScan) { } } } - StorageBucketHashType function = StorageBucketHashType.STORAGE_BUCKET_CRC32; - if (hmsExternalTable.isSparkBucketedTable()) { - function = StorageBucketHashType.STORAGE_BUCKET_SPARK_MURMUR32; - } + return new DistributionSpecHash(hashColumns, DistributionSpecHash.ShuffleType.NATURAL, - fileScan.getTable().getId(), -1, Collections.emptySet(), function); + fileScan.getTable().getId(), -1, Collections.emptySet(), + StorageBucketHashType.STORAGE_BUCKET_SPARK_MURMUR32); } return DistributionSpecStorageAny.INSTANCE; diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java index 3ba8efa48a29a0..7b6575d66f64dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/DistributedPlanner.java @@ -41,6 +41,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.source.HiveScanNode; +import org.apache.doris.nereids.properties.DistributionSpecHash; import org.apache.doris.qe.ConnectContext; import org.apache.doris.thrift.THashType; import org.apache.doris.thrift.TPartitionType; @@ -339,6 +340,7 @@ private PlanFragment createHashJoinFragment( Ref hashType = Ref.from(THashType.CRC32); if (canBucketShuffleJoin(node, leftChildFragment, rhsPartitionExprs, hashType)) { node.setDistributionMode(HashJoinNode.DistributionMode.BUCKET_SHUFFLE); + node.setHashType(DistributionSpecHash.StorageBucketHashType.fromThrift(hashType.value)); DataPartition rhsJoinPartition = new DataPartition(TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED, rhsPartitionExprs, hashType.value); @@ -658,7 +660,8 @@ private boolean canBucketShuffleJoinForHive(HashJoinNode node, HiveScanNode left HMSExternalTable leftTable = leftScanNode.getHiveTable(); DistributionInfo leftDistribution = leftTable.getDefaultDistributionInfo(); - if (leftDistribution == null || !(leftDistribution instanceof HiveExternalDistributionInfo)) { + if (leftDistribution == null || !(leftDistribution instanceof HiveExternalDistributionInfo) + || !ConnectContext.get().getSessionVariable().isEnableBucketShuffleJoin()) { return false; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java index cb6628b01c556b..a13f830692f385 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java @@ -30,6 +30,7 @@ import org.apache.doris.statistics.StatsRecursiveDerive; import org.apache.doris.thrift.TExchangeNode; import org.apache.doris.thrift.TExplainLevel; +import org.apache.doris.thrift.THashType; import org.apache.doris.thrift.TPartitionType; import org.apache.doris.thrift.TPlanNode; import org.apache.doris.thrift.TPlanNodeType; @@ -67,6 +68,7 @@ public class ExchangeNode extends PlanNode { private boolean isRightChildOfBroadcastHashJoin = false; private TPartitionType partitionType; + private THashType hashType; /** * use for Nereids only. @@ -168,6 +170,10 @@ public void setMergeInfo(SortInfo info) { this.planNodeName = "V" + MERGING_EXCHANGE_NODE; } + public void setHashType(THashType hashType) { + this.hashType = hashType; + } + @Override protected void toThrift(TPlanNode msg) { // If this fragment has another scan node, this exchange node is serial or not should be decided by the scan @@ -182,6 +188,9 @@ protected void toThrift(TPlanNode msg) { if (mergeInfo != null) { msg.exchange_node.setSortInfo(mergeInfo.toThrift()); } + if (hashType != null) { + msg.exchange_node.setHashType(hashType); + } msg.exchange_node.setOffset(offset); msg.exchange_node.setPartitionType(partitionType); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/HashJoinNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/HashJoinNode.java index c3cbf2afce15ac..685e07b13b3991 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/HashJoinNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/HashJoinNode.java @@ -37,6 +37,7 @@ import org.apache.doris.common.CheckedMath; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; +import org.apache.doris.nereids.properties.DistributionSpecHash; import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.statistics.StatisticalType; import org.apache.doris.thrift.TEqJoinCondition; @@ -79,6 +80,7 @@ public class HashJoinNode extends JoinNodeBase { private List markJoinConjuncts; private DistributionMode distrMode; + private DistributionSpecHash.StorageBucketHashType hashType; private boolean isColocate = false; //the flag for colocate join private String colocateReason = ""; // if can not do colocate join, set reason here @@ -249,6 +251,10 @@ public void setColocate(boolean colocate, String reason) { colocateReason = reason; } + public void setHashType(DistributionSpecHash.StorageBucketHashType hashType) { + this.hashType = hashType; + } + /** * Calculate the slots output after going through the hash table in the hash join node. * The most essential difference between 'hashOutputSlots' and 'outputSlots' is that @@ -817,6 +823,9 @@ protected void toThrift(TPlanNode msg) { } } msg.hash_join_node.setDistType(isColocate ? TJoinDistributionType.COLOCATE : distrMode.toThrift()); + if (hashType != null) { + msg.hash_join_node.setHashType(hashType.toThrift()); + } msg.hash_join_node.setUseSpecificProjections(useSpecificProjections); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 08c442c5877566..164453339ae754 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -37,8 +37,8 @@ import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.ExternalScanNode; import org.apache.doris.datasource.FileQueryScanNode; -import org.apache.doris.datasource.hive.source.HiveScanNode; import org.apache.doris.datasource.hive.HMSTransaction; +import org.apache.doris.datasource.hive.source.HiveScanNode; import org.apache.doris.datasource.iceberg.IcebergTransaction; import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.metric.MetricRepo; @@ -2160,7 +2160,7 @@ protected void computeScanRangeAssignment() throws Exception { scanNode.getFragment().getPlanRoot()) && (scanNode instanceof OlapScanNode); boolean fragmentContainsBucketShuffleJoin = bucketShuffleJoinController .isBucketShuffleJoin(scanNode.getFragmentId().asInt(), scanNode.getFragment().getPlanRoot()) - && (scanNode instanceof OlapScanNode); + && (scanNode instanceof OlapScanNode || scanNode instanceof HiveScanNode); // A fragment may contain both colocate join and bucket shuffle join // on need both compute scanRange to init basic data for query coordinator @@ -2561,7 +2561,7 @@ class BucketShuffleJoinController { private final Map> fragmentIdToSeqToAddressMap = Maps.newHashMap(); // fragment_id -> < be_id -> bucket_count > - private final Map> fragmentIdToBuckendIdBucketCountMap = Maps.newHashMap(); + private final Map> fragmentIdToBackendIdBucketCountMap = Maps.newHashMap(); // fragment_id -> bucket_num protected final Map fragmentIdToBucketNumMap = Maps.newHashMap(); @@ -2616,30 +2616,30 @@ private void getExecHostPortForFragmentIDAndBucketSeq(TScanRangeLocations seqLoc PlanFragmentId fragmentId, Integer bucketSeq, ImmutableMap idToBackend, Map addressToBackendID, Map replicaNumPerHost) throws Exception { - Map buckendIdToBucketCountMap = fragmentIdToBuckendIdBucketCountMap.get(fragmentId); + Map backendIdToBucketCountMap = fragmentIdToBackendIdBucketCountMap.get(fragmentId); int maxBucketNum = Integer.MAX_VALUE; - long buckendId = Long.MAX_VALUE; + long backendId = Long.MAX_VALUE; Long minReplicaNum = Long.MAX_VALUE; for (TScanRangeLocation location : seqLocation.locations) { - if (buckendIdToBucketCountMap.getOrDefault(location.backend_id, 0) < maxBucketNum) { - maxBucketNum = buckendIdToBucketCountMap.getOrDefault(location.backend_id, 0); - buckendId = location.backend_id; + if (backendIdToBucketCountMap.getOrDefault(location.backend_id, 0) < maxBucketNum) { + maxBucketNum = backendIdToBucketCountMap.getOrDefault(location.backend_id, 0); + backendId = location.backend_id; minReplicaNum = replicaNumPerHost.get(location.server); - } else if (buckendIdToBucketCountMap.getOrDefault(location.backend_id, 0) == maxBucketNum + } else if (backendIdToBucketCountMap.getOrDefault(location.backend_id, 0) == maxBucketNum && replicaNumPerHost.get(location.server) < minReplicaNum) { - buckendId = location.backend_id; + backendId = location.backend_id; minReplicaNum = replicaNumPerHost.get(location.server); } } Reference backendIdRef = new Reference<>(); - TNetworkAddress execHostPort = SimpleScheduler.getHost(buckendId, + TNetworkAddress execHostPort = SimpleScheduler.getHost(backendId, seqLocation.locations, idToBackend, backendIdRef); - //the backend with buckendId is not alive, chose another new backend - if (backendIdRef.getRef() != buckendId) { - buckendIdToBucketCountMap.put(backendIdRef.getRef(), - buckendIdToBucketCountMap.getOrDefault(backendIdRef.getRef(), 0) + 1); - } else { //the backend with buckendId is alive, update buckendIdToBucketCountMap directly - buckendIdToBucketCountMap.put(buckendId, buckendIdToBucketCountMap.getOrDefault(buckendId, 0) + 1); + //the backend with backendId is not alive, chose another new backend + if (backendIdRef.getRef() != backendId) { + backendIdToBucketCountMap.put(backendIdRef.getRef(), + backendIdToBucketCountMap.getOrDefault(backendIdRef.getRef(), 0) + 1); + } else { //the backend with backendId is alive, update backendIdToBucketCountMap directly + backendIdToBucketCountMap.put(backendId, backendIdToBucketCountMap.getOrDefault(backendId, 0) + 1); } for (TScanRangeLocation location : seqLocation.locations) { replicaNumPerHost.put(location.server, replicaNumPerHost.get(location.server) - 1); @@ -2671,7 +2671,7 @@ private void computeScanRangeAssignmentByBucketForOlap( fragmentIdToBucketNumMap.put(scanNode.getFragmentId(), bucketNum); fragmentIdToSeqToAddressMap.put(scanNode.getFragmentId(), new HashMap<>()); fragmentIdBucketSeqToScanRangeMap.put(scanNode.getFragmentId(), new BucketSeqToScanRange()); - fragmentIdToBuckendIdBucketCountMap.put(scanNode.getFragmentId(), new HashMap<>()); + fragmentIdToBackendIdBucketCountMap.put(scanNode.getFragmentId(), new HashMap<>()); scanNode.getFragment().setBucketNum(bucketNum); } Map bucketSeqToAddress @@ -2716,7 +2716,8 @@ private void computeScanRangeAssignmentByBucketForHive( fragmentIdToBucketNumMap.put(scanNode.getFragmentId(), bucketNum); fragmentIdToSeqToAddressMap.put(scanNode.getFragmentId(), new HashMap<>()); fragmentIdBucketSeqToScanRangeMap.put(scanNode.getFragmentId(), new BucketSeqToScanRange()); - fragmentIdToBuckendIdBucketCountMap.put(scanNode.getFragmentId(), new HashMap<>()); + fragmentIdToBackendIdBucketCountMap.put(scanNode.getFragmentId(), new HashMap<>()); + scanNode.getFragment().setBucketNum(bucketNum); } Map bucketSeqToAddress = fragmentIdToSeqToAddressMap.get(scanNode.getFragmentId()); @@ -2848,13 +2849,10 @@ private void assignScanRanges(PlanFragmentId fragmentId, int parallelExecInstanc Preconditions.checkArgument(node.isPresent()); FInstanceExecParam instanceParamToScan = node.get().isSerialOperator() ? firstInstanceParam : instanceParam; - if (!instanceParamToScan.perNodeScanRanges.containsKey(nodeScanRange.getKey())) { - range.put(nodeScanRange.getKey(), Lists.newArrayList()); - instanceParamToScan.perNodeScanRanges - .put(nodeScanRange.getKey(), Lists.newArrayList()); - } - range.get(nodeScanRange.getKey()).addAll(nodeScanRange.getValue()); - instanceParamToScan.perNodeScanRanges.get(nodeScanRange.getKey()) + range.computeIfAbsent(nodeScanRange.getKey(), ArrayList::new) + .addAll(nodeScanRange.getValue()); + instanceParamToScan.perNodeScanRanges + .computeIfAbsent(nodeScanRange.getKey(), ArrayList::new) .addAll(nodeScanRange.getValue()); } } @@ -3354,10 +3352,28 @@ public void appendScanRange(StringBuilder sb, List params) { } TEsScanRange esScanRange = range.getScanRange().getEsScanRange(); if (esScanRange != null) { + if (idx++ != 0) { + sb.append(","); + } + if (idx++ != 0) { + sb.append(","); + } sb.append("{ index=").append(esScanRange.getIndex()) .append(", shardid=").append(esScanRange.getShardId()) .append("}"); } + TExternalScanRange extScanRange = range.getScanRange().getExtScanRange(); + if (extScanRange != null) { + TFileScanRange fileScanRange = extScanRange.getFileScanRange(); + if (fileScanRange != null) { + if (idx++ != 0) { + sb.append(","); + } + sb.append("{path=") + .append(fileScanRange.getRanges().get(0).getPath()) + .append("}"); + } + } } sb.append("]"); } @@ -3374,10 +3390,10 @@ public void appendTo(StringBuilder sb) { } TNetworkAddress address = instanceExecParams.get(i).host; Map> scanRanges = - scanRangeAssignment.get(address); + instanceExecParams.get(i).perNodeScanRanges; sb.append("{"); sb.append("id=").append(DebugUtil.printId(instanceExecParams.get(i).instanceId)); - sb.append(",host=").append(instanceExecParams.get(i).host); + sb.append(",host=").append(address); if (scanRanges == null) { sb.append("}"); continue; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 18289a23bd4538..9bed20848a3191 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -710,6 +710,8 @@ public class SessionVariable implements Serializable, Writable { public static final String ENABLE_TEXT_VALIDATE_UTF8 = "enable_text_validate_utf8"; + public static final String ENABLE_SPARK_BUCKET_SHUFFLE = "enable_spark_bucket_shuffle"; + /** * If set false, user couldn't submit analyze SQL and FE won't allocate any related resources. */ @@ -2428,6 +2430,17 @@ public boolean isEnableSortSpill() { return enableSortSpill; } + @VariableMgr.VarAttr(name = ENABLE_SPARK_BUCKET_SHUFFLE) + public boolean enableSparkShuffle = false; + + public boolean isEnableSparkShuffle() { + return enableSparkShuffle; + } + + public void setEnableSparkShuffle(boolean enableSparkShuffle) { + this.enableSparkShuffle = enableSparkShuffle; + } + // If this fe is in fuzzy mode, then will use initFuzzyModeVariables to generate some variables, // not the default value set in the code. @SuppressWarnings("checkstyle:Indentation") diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index 70c6722b9d8580..2bda1e424da17a 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -866,6 +866,7 @@ struct THashJoinNode { 13: optional list mark_join_conjuncts // use_specific_projections true, if output exprssions is denoted by srcExprList represents, o.w. PlanNode.projections 14: optional bool use_specific_projections + 15: optional Partitions.THashType hash_type } struct TNestedLoopJoinNode { @@ -1171,6 +1172,7 @@ struct TExchangeNode { 3: optional i64 offset // Shuffle partition type 4: optional Partitions.TPartitionType partition_type + 5: optional Partitions.THashType hash_type } struct TOlapRewriteNode { diff --git a/regression-test/suites/external_table_p0/hive/test_hive_spark_clustered_table.groovy b/regression-test/suites/external_table_p0/hive/test_hive_spark_clustered_table.groovy index bf7f5c1794a96f..0751d1f6e65c48 100644 --- a/regression-test/suites/external_table_p0/hive/test_hive_spark_clustered_table.groovy +++ b/regression-test/suites/external_table_p0/hive/test_hive_spark_clustered_table.groovy @@ -51,6 +51,7 @@ suite("test_hive_spark_clustered_table", "p0,external,hive,external_docker,exter sql """use `${catalog_name}`.`default`""" sql """set enable_fallback_to_original_planner=false;""" + sql """SET enable_spark_bucket_shuffle=true""" q01() diff --git a/regression-test/suites/nereids_p0/join/bucket_shuffle_join.groovy b/regression-test/suites/nereids_p0/join/bucket_shuffle_join.groovy index dd6a00b3123e74..e85a2bbc088964 100644 --- a/regression-test/suites/nereids_p0/join/bucket_shuffle_join.groovy +++ b/regression-test/suites/nereids_p0/join/bucket_shuffle_join.groovy @@ -20,6 +20,7 @@ suite("bucket-shuffle-join") { sql "SET enable_fallback_to_original_planner=false" sql 'SET be_number_for_test=1' sql 'SET parallel_pipeline_task_num=1' + sql 'SET enable_spark_bucket_shuffle=true' order_qt_test_bucket """ select * from test_bucket_shuffle_join where rectime="2021-12-01 00:00:00" and id in (select k1 from test_join where k1 in (1,2)) """