Skip to content

Commit

Permalink
update
Browse files Browse the repository at this point in the history
  • Loading branch information
Gabriel39 committed Sep 26, 2024
1 parent e35dc83 commit 71361c6
Showing 1 changed file with 5 additions and 2 deletions.
7 changes: 5 additions & 2 deletions be/src/pipeline/exec/hashjoin_build_sink.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,9 @@ Status HashJoinBuildSinkLocalState::open(RuntimeState* state) {
}

Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_status) {
if (_closed) {
return Status::OK();
}
auto p = _parent->cast<HashJoinBuildSinkOperatorX>();
Defer defer {[&]() {
if (_should_build_hash_table && p._shared_hashtable_controller) {
Expand All @@ -119,7 +122,7 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu
}};

if (!_runtime_filter_slots || _runtime_filters.empty() || state->is_cancelled() || !_eos) {
return Status::OK();
return Base::close(state, exec_status);
}
auto* block = _shared_state->build_block.get();
uint64_t hash_table_size = block ? block->rows() : 0;
Expand All @@ -137,7 +140,7 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu

SCOPED_TIMER(_publish_runtime_filter_timer);
RETURN_IF_ERROR(_runtime_filter_slots->publish(!_should_build_hash_table));
return Status::OK();
return Base::close(state, exec_status);
}

bool HashJoinBuildSinkLocalState::build_unique() const {
Expand Down

0 comments on commit 71361c6

Please sign in to comment.