Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Revert "[Improvement](sink) optimization for parallel result sink (#3… #36628

Merged
merged 1 commit into from
Jun 21, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 2 additions & 3 deletions be/src/pipeline/exec/result_file_sink_operator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -99,8 +99,7 @@ Status ResultFileSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& i
if (p._is_top_sink) {
// create sender
RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(
state->fragment_instance_id(), p._buf_size, &_sender, state->execution_timeout(),
state->batch_size()));
state->fragment_instance_id(), p._buf_size, &_sender, state->execution_timeout()));
// create writer
_writer.reset(new (std::nothrow) vectorized::VFileResultWriter(
p._file_opts.get(), p._storage_type, state->fragment_instance_id(),
Expand Down Expand Up @@ -176,7 +175,7 @@ Status ResultFileSinkLocalState::close(RuntimeState* state, Status exec_status)
// close sender, this is normal path end
if (_sender) {
_sender->update_return_rows(_writer == nullptr ? 0 : _writer->get_written_rows());
RETURN_IF_ERROR(_sender->close(state->fragment_instance_id(), final_status));
RETURN_IF_ERROR(_sender->close(final_status));
}
state->exec_env()->result_mgr()->cancel_at_time(
time(nullptr) + config::result_buffer_cancelled_interval_time,
Expand Down
2 changes: 1 addition & 1 deletion be/src/pipeline/exec/result_file_sink_operator.h
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ class ResultFileSinkOperatorX final : public DataSinkOperatorX<ResultFileSinkLoc

// Owned by the RuntimeState.
RowDescriptor _output_row_descriptor;
int _buf_size = 4096; // Allocated from _pool
int _buf_size = 1024; // Allocated from _pool
bool _is_top_sink = true;
std::string _header;
std::string _header_type;
Expand Down
13 changes: 6 additions & 7 deletions be/src/pipeline/exec/result_sink_operator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -49,10 +49,10 @@ Status ResultSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info)
_sender = _parent->cast<ResultSinkOperatorX>()._sender;
} else {
RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(
fragment_instance_id, RESULT_SINK_BUFFER_SIZE, &_sender, state->execution_timeout(),
state->batch_size()));
state->fragment_instance_id(), RESULT_SINK_BUFFER_SIZE, &_sender,
state->execution_timeout()));
}
_sender->set_dependency(fragment_instance_id, _dependency->shared_from_this());
_sender->set_dependency(_dependency->shared_from_this());
return Status::OK();
}

Expand Down Expand Up @@ -122,8 +122,7 @@ Status ResultSinkOperatorX::prepare(RuntimeState* state) {

if (state->query_options().enable_parallel_result_sink) {
RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(
state->query_id(), RESULT_SINK_BUFFER_SIZE, &_sender, state->execution_timeout(),
state->batch_size()));
state->query_id(), RESULT_SINK_BUFFER_SIZE, &_sender, state->execution_timeout()));
}
return Status::OK();
}
Expand All @@ -140,7 +139,7 @@ Status ResultSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block,
if (_fetch_option.use_two_phase_fetch && block->rows() > 0) {
RETURN_IF_ERROR(_second_phase_fetch_data(state, block));
}
RETURN_IF_ERROR(local_state._writer->write(state, *block));
RETURN_IF_ERROR(local_state._writer->write(*block));
if (_fetch_option.use_two_phase_fetch) {
// Block structure may be changed by calling _second_phase_fetch_data().
// So we should clear block in case of unmatched columns
Expand Down Expand Up @@ -186,7 +185,7 @@ Status ResultSinkLocalState::close(RuntimeState* state, Status exec_status) {
if (_writer) {
_sender->update_return_rows(_writer->get_written_rows());
}
RETURN_IF_ERROR(_sender->close(state->fragment_instance_id(), final_status));
RETURN_IF_ERROR(_sender->close(final_status));
}
state->exec_env()->result_mgr()->cancel_at_time(
time(nullptr) + config::result_buffer_cancelled_interval_time,
Expand Down
2 changes: 1 addition & 1 deletion be/src/pipeline/exec/result_sink_operator.h
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ struct ResultFileOptions {
}
};

constexpr int RESULT_SINK_BUFFER_SIZE = 4096 * 8;
constexpr int RESULT_SINK_BUFFER_SIZE = 4096;

class ResultSinkLocalState final : public PipelineXSinkLocalState<BasicSharedState> {
ENABLE_FACTORY_CREATOR(ResultSinkLocalState);
Expand Down
22 changes: 15 additions & 7 deletions be/src/pipeline/local_exchange/local_exchanger.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -200,9 +200,8 @@ Status PassthroughExchanger::sink(RuntimeState* state, vectorized::Block* in_blo
}
new_block.swap(*in_block);
auto channel_id = (local_state._channel_id++) % _num_partitions;
size_t allocated_bytes = new_block.allocated_bytes();
local_state._shared_state->add_mem_usage(channel_id, new_block.allocated_bytes());
if (_data_queue[channel_id].enqueue(std::move(new_block))) {
local_state._shared_state->add_mem_usage(channel_id, allocated_bytes);
local_state._shared_state->set_ready_to_read(channel_id);
}

Expand All @@ -221,16 +220,25 @@ void PassthroughExchanger::close(LocalExchangeSourceLocalState& local_state) {
Status PassthroughExchanger::get_block(RuntimeState* state, vectorized::Block* block, bool* eos,
LocalExchangeSourceLocalState& local_state) {
vectorized::Block next_block;
bool all_finished = _running_sink_operators == 0;
if (_data_queue[local_state._channel_id].try_dequeue(next_block)) {
if (_running_sink_operators == 0) {
if (_data_queue[local_state._channel_id].try_dequeue(next_block)) {
block->swap(next_block);
local_state._shared_state->sub_mem_usage(local_state._channel_id,
block->allocated_bytes());
if (_free_block_limit == 0 ||
_free_blocks.size_approx() < _free_block_limit * _num_sources) {
_free_blocks.enqueue(std::move(next_block));
}
} else {
*eos = true;
}
} else if (_data_queue[local_state._channel_id].try_dequeue(next_block)) {
block->swap(next_block);
local_state._shared_state->sub_mem_usage(local_state._channel_id, block->allocated_bytes());
if (_free_block_limit == 0 ||
_free_blocks.size_approx() < _free_block_limit * _num_sources) {
_free_blocks.enqueue(std::move(next_block));
}
} else if (all_finished) {
*eos = true;
local_state._shared_state->sub_mem_usage(local_state._channel_id, block->allocated_bytes());
} else {
COUNTER_UPDATE(local_state._get_block_failed_counter, 1);
local_state._dependency->block();
Expand Down
Loading
Loading