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/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..9078731ee438ec 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() \ + 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 { 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..16a3fc256559b1 100644 --- a/be/src/vec/columns/column_array.cpp +++ b/be/src/vec/columns/column_array.cpp @@ -319,6 +319,57 @@ 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 (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..78c0d9162bbc2c 100644 --- a/be/src/vec/columns/column_decimal.cpp +++ b/be/src/vec/columns/column_decimal.cpp @@ -183,6 +183,54 @@ 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), hash); + } 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), hash); + } 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() + } 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..a095620ddfe04d 100644 --- a/be/src/vec/columns/column_map.cpp +++ b/be/src/vec/columns/column_map.cpp @@ -343,6 +343,37 @@ 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 (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 +409,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..49854b30163fc5 100644 --- a/be/src/vec/columns/column_nullable.cpp +++ b/be/src/vec/columns/column_nullable.cpp @@ -88,6 +88,22 @@ 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(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); + } + } + 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 +132,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(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(hashes[i]); + } + } + 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..fd6f09d88524ea 100644 --- a/be/src/vec/columns/column_string.cpp +++ b/be/src/vec/columns/column_string.cpp @@ -286,6 +286,28 @@ void ColumnStr::update_crcs_with_value(uint32_t* __restrict hashes, doris::Pr } } +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; + 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, 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, hashes[i]); + } + } + } +} + 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..dd4ca1e69ed0b4 100644 --- a/be/src/vec/columns/column_string.h +++ b/be/src/vec/columns/column_string.h @@ -427,6 +427,23 @@ 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, 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, hash); + } + } + } + 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 +457,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..1c75f26ec37ed9 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() + } 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, hashes[i]); + }; + + 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() + } + } +} + 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..7297f7a6efb05c 100644 --- a/be/src/vec/columns/column_vector.h +++ b/be/src/vec/columns/column_vector.h @@ -306,12 +306,32 @@ 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 (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..5f4f651c498706 100644 --- a/be/src/vec/runtime/partitioner.cpp +++ b/be/src/vec/runtime/partitioner.cpp @@ -17,18 +17,17 @@ #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 Crc32HashPartitioner::do_partitioning(RuntimeState* state, Block* block) const { - size_t rows = block->rows(); +template +Status Partitioner::do_partitioning(RuntimeState* state, + Block* block) const { + int rows = block->rows(); if (rows > 0) { auto column_to_keep = block->columns(); @@ -37,7 +36,7 @@ Status Crc32HashPartitioner::do_partitioning(RuntimeState* state, Bl 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) { @@ -60,24 +59,52 @@ Status Crc32HashPartitioner::do_partitioning(RuntimeState* state, Bl 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())); } +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) { - 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(); } +template +Status Murmur32HashPartitioner::clone(RuntimeState* state, + std::unique_ptr& partitioner) { + 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++) { + RETURN_IF_ERROR(Base::_partition_expr_ctxs[i]->clone( + state, new_partitioner->_partition_expr_ctxs[i])); + } + 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; } // namespace doris::vectorized diff --git a/be/src/vec/runtime/partitioner.h b/be/src/vec/runtime/partitioner.h index 53d8b84d09c752..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 { @@ -111,5 +113,42 @@ struct SpillPartitionChannelIds { return ((l >> 16) | (l << 16)) % r; } }; + +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(size_t 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; + int32_t _get_default_seed() const override; +}; + #include "common/compile_check_end.h" } // namespace doris::vectorized 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..d30d0f2e36cbfb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveExternalDistributionInfo.java @@ -0,0 +1,80 @@ +// 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(int bucketNum, List distributionColumns, int bucketingVersion) { + super(bucketNum, distributionColumns); + this.bucketingVersion = 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..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 @@ -248,6 +268,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 +526,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..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,16 @@ 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.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; @@ -64,7 +68,9 @@ 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; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; @@ -86,6 +92,8 @@ 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; @@ -274,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(); @@ -319,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(); @@ -353,10 +375,12 @@ public void createScanRangeLocations() throws UserException { tSource.setSplitSourceId(splitSource.getUniqueId()); tSource.setNumSplits(numSplitsPerBE); curLocations.getScanRange().getExtScanRange().getFileScanRange().setSplitSource(tSource); + 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. // Each backend only starts up one ScanNode instance. // However, even one ScanNode instance can provide maximum scanning concurrency. @@ -373,14 +397,33 @@ public void createScanRangeLocations() throws UserException { 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()); } } @@ -397,6 +440,7 @@ public void createScanRangeLocations() throws UserException { private TScanRangeLocations splitToScanRange( Backend backend, + Integer bucketNum, Map locationProperties, Split split, List pathPartitionKeys) throws UserException { @@ -413,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); @@ -451,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; } @@ -592,6 +656,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/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 71c7308b079866..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 @@ -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; @@ -34,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; @@ -59,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; @@ -73,6 +79,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; @@ -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 HiveBucketType hiveBucketType = HiveBucketType.NONE; 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,10 @@ public boolean isHoodieCowTable() { || (params != null && "COPY_ON_WRITE".equalsIgnoreCase(params.get("flink.table.type"))); } + public boolean isSparkBucketedTable() { + return bucketColumns != null && !bucketColumns.isEmpty() && hiveBucketType == HiveBucketType.SPARK; + } + /** * 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. @@ -545,10 +577,73 @@ 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)); } + private void initBucketingColumns(List columns) { + List bucketCols = new ArrayList<>(5); + int numBuckets = getBucketColumns(bucketCols); + if (bucketCols.isEmpty() || hiveBucketType != HiveBucketType.SPARK) { + 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(); + hiveBucketType = HiveBucketType.HIVE; + } 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) { + hiveBucketType = HiveBucketType.SPARK; + } + } + + return numBuckets; + } + private Optional getHudiSchema() { org.apache.avro.Schema hudiSchema = HiveMetaStoreClientHelper.getHudiTableSchema(this); List tmpSchema = Lists.newArrayListWithCapacity(hudiSchema.getFields().size()); @@ -559,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); @@ -577,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)); } @@ -661,6 +756,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 +890,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..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 @@ -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, @@ -96,6 +102,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 +407,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..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 @@ -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.isSparkBucketedTable() && ConnectContext.get().getSessionVariable().isEnableSparkShuffle()) { + 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.isSparkBucketedTable() && ConnectContext.get().getSessionVariable().isEnableSparkShuffle() + && 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..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 @@ -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; @@ -347,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) { @@ -556,7 +558,7 @@ public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTransla SessionVariable sv = ConnectContext.get().getSessionVariable(); // TODO(cmy): determine the needCheckColumnPriv param - ScanNode scanNode; + FileQueryScanNode scanNode; if (table instanceof HMSExternalTable) { switch (((HMSExternalTable) table).getDlaType()) { case ICEBERG: @@ -588,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); } @@ -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, ((FileQueryScanNode) scanNode).getHashType()); + } + // Create PlanFragment PlanFragment planFragment = createPlanFragment(scanNode, dataPartition, fileScan); context.addPlanFragment(planFragment); updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), fileScan); @@ -1418,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); } @@ -2604,7 +2616,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 +2643,10 @@ private DataPartition toDataPartition(DistributionSpec distributionSpec, } } TPartitionType partitionType; + THashType hashType = THashType.XXHASH64; switch (distributionSpecHash.getShuffleType()) { case STORAGE_BUCKETED: + hashType = distributionSpecHash.getShuffleFunction().toThrift(); partitionType = TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED; break; case EXECUTION_BUCKETED: @@ -2643,7 +2657,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..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 @@ -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()); } /** @@ -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 b821ff0de87ae0..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())); + 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 6ab8e054f8aaef..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 @@ -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; @@ -54,6 +55,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 +73,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 +104,7 @@ public DistributionSpecHash(List orderedShuffledColumns, ShuffleType shu } this.equivalenceExprIds = equivalenceExprIdsBuilder.build(); this.exprIdToEquivalenceSet = exprIdToEquivalenceSetBuilder.buildKeepingLast(); + this.storageBucketHashType = storageBucketHashType; } /** @@ -101,7 +114,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 +122,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 +134,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 +154,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 +189,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 +245,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 +291,7 @@ public DistributionSpec project(Map projections, } } return new DistributionSpecHash(orderedShuffledColumns, shuffleType, tableId, selectedIndexId, partitionIds, - equivalenceExprIds, exprIdToEquivalenceSet); + equivalenceExprIds, exprIdToEquivalenceSet, storageBucketHashType); } @Override @@ -281,12 +300,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 +335,48 @@ public enum ShuffleType { STORAGE_BUCKETED, } + /** + * 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, + // 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: + return THashType.CRC32; + case STORAGE_BUCKET_SPARK_MURMUR32: + return THashType.SPARK_MURMUR32; + case STORAGE_BUCKET_XXHASH64: + 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/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..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 @@ -17,12 +17,29 @@ 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.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; + +import java.util.Collections; +import java.util.List; import java.util.Optional; /** @@ -36,7 +53,7 @@ public Rule build() { fileScan.getRelationId(), fileScan.getTable(), fileScan.getQualifier(), - DistributionSpecAny.INSTANCE, + convertDistribution(fileScan), Optional.empty(), fileScan.getLogicalProperties(), fileScan.getSelectedPartitions(), @@ -44,4 +61,42 @@ 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; + 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 && isSelectUnpartitioned + && ConnectContext.get().getSessionVariable().isEnableSparkShuffle()) { + 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()); + } + } + } + + return new DistributionSpecHash(hashColumns, DistributionSpecHash.ShuffleType.NATURAL, + 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/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..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 @@ -33,17 +33,23 @@ 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.nereids.properties.DistributionSpecHash; 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 +291,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 +337,13 @@ 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); + node.setHashType(DistributionSpecHash.StorageBucketHashType.fromThrift(hashType.value)); 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 +613,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; } @@ -614,9 +627,8 @@ private boolean canBucketShuffleJoin(HashJoinNode node, PlanFragment leftChildFr } PlanNode leftRoot = leftChildFragment.getPlanRoot(); - // 1.leftRoot be OlapScanNode - if (leftRoot instanceof OlapScanNode) { - return canBucketShuffleJoin(node, leftRoot, rhsHashExprs); + if (leftRoot instanceof ScanNode) { + return canBucketShuffleJoin(node, (ScanNode) leftRoot, rhsHashExprs, hashType); } // 2.leftRoot be hashjoin node @@ -624,18 +636,94 @@ private boolean canBucketShuffleJoin(HashJoinNode node, PlanFragment leftChildFr while (leftRoot instanceof HashJoinNode) { leftRoot = leftRoot.getChild(0); } - if (leftRoot instanceof OlapScanNode) { - return canBucketShuffleJoin(node, leftRoot, rhsHashExprs); + if (leftRoot instanceof ScanNode) { + canBucketShuffleJoin(node, (ScanNode) leftRoot, rhsHashExprs, hashType); } } return false; } + 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(); + + DistributionInfo leftDistribution = leftTable.getDefaultDistributionInfo(); + if (leftDistribution == null || !(leftDistribution instanceof HiveExternalDistributionInfo) + || !ConnectContext.get().getSessionVariable().isEnableBucketShuffleJoin()) { + 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 canBucketShuffleJoinForOlap(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/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/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..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 @@ -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; @@ -37,6 +38,7 @@ import org.apache.doris.datasource.ExternalScanNode; import org.apache.doris.datasource.FileQueryScanNode; 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; @@ -2158,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 @@ -2167,7 +2169,7 @@ protected void computeScanRangeAssignment() throws Exception { replicaNumPerHost, isEnableOrderedLocations); } if (fragmentContainsBucketShuffleJoin) { - bucketShuffleJoinController.computeScanRangeAssignmentByBucket((OlapScanNode) scanNode, + bucketShuffleJoinController.computeScanRangeAssignmentByBucket(scanNode, idToBackend, addressToBackendID, replicaNumPerHost); } if (!(fragmentContainsColocateJoin || fragmentContainsBucketShuffleJoin)) { @@ -2559,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(); @@ -2614,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); @@ -2646,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 { @@ -2656,7 +2671,52 @@ private void computeScanRangeAssignmentByBucket( 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()); + 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 computeScanRangeAssignmentByBucketForHive( + final HiveScanNode scanNode, ImmutableMap idToBackend, + Map addressToBackendID, + Map replicaNumPerHost) throws Exception { + if (!fragmentIdToSeqToAddressMap.containsKey(scanNode.getFragmentId())) { + int bucketNum = 0; + if (scanNode.getHiveTable().isSparkBucketedTable()) { + 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()); + fragmentIdToBackendIdBucketCountMap.put(scanNode.getFragmentId(), new HashMap<>()); scanNode.getFragment().setBucketNum(bucketNum); } Map bucketSeqToAddress @@ -2789,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()); } } @@ -3295,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("]"); } @@ -3315,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/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/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/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..0751d1f6e65c48 --- /dev/null +++ b/regression-test/suites/external_table_p0/hive/test_hive_spark_clustered_table.groovy @@ -0,0 +1,68 @@ +// 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;""" + sql """SET enable_spark_bucket_shuffle=true""" + + 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..e85a2bbc088964 --- /dev/null +++ b/regression-test/suites/nereids_p0/join/bucket_shuffle_join.groovy @@ -0,0 +1,84 @@ +// 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' + 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)) + """ + + 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" + } + +}