Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
50 commits
Select commit Hold shift + click to select a range
4ad09d0
resolve conflict
dqhl76 Oct 16, 2025
2b238d4
refactor: add settings
dqhl76 Sep 27, 2025
e2af595
refactor: before repartition
dqhl76 Oct 8, 2025
b4cdf96
save
dqhl76 Oct 9, 2025
7ab0024
save
dqhl76 Oct 9, 2025
9536375
add back log
dqhl76 Oct 9, 2025
0c95f5a
remove useless
dqhl76 Oct 9, 2025
4eeacf2
extract some from final
dqhl76 Oct 12, 2025
4ef788a
save
dqhl76 Oct 12, 2025
1badee1
save
dqhl76 Oct 12, 2025
b05d942
fix: partition id
dqhl76 Oct 12, 2025
dc29ca4
fix: compile
dqhl76 Oct 13, 2025
1d4def4
save
dqhl76 Oct 13, 2025
0914c17
save
dqhl76 Oct 16, 2025
2b52345
refactor
dqhl76 Oct 16, 2025
74c21bf
refactor
dqhl76 Oct 16, 2025
34d315d
save
dqhl76 Oct 19, 2025
286b10b
this commit add Debug trait for AggregateMeta, should revert before m…
dqhl76 Oct 19, 2025
58f1ec3
fix: reset aggregate status
dqhl76 Oct 19, 2025
13fda1d
fix: fix dispatcher cannot finish caused hang
dqhl76 Oct 19, 2025
3841767
clean, start spill and restore refactor
dqhl76 Oct 19, 2025
c8ad7b4
check memory pressure
dqhl76 Oct 19, 2025
f2e59f2
feat: add support for recursive schedule
dqhl76 Oct 19, 2025
82389c2
feat: add support for recursive spill
dqhl76 Oct 19, 2025
2b5d17f
make lint
dqhl76 Oct 19, 2025
547f810
partition stream
dqhl76 Oct 20, 2025
d9fa9a2
Revert "this commit add Debug trait for AggregateMeta, should revert …
dqhl76 Oct 20, 2025
15d8a8e
chore: clean debug
dqhl76 Oct 20, 2025
0905560
refactor: merge dispatcher with bucket scheduler
dqhl76 Oct 20, 2025
43cd5fe
add back aggregate
dqhl76 Oct 20, 2025
b294efd
add back spill restore
dqhl76 Oct 21, 2025
ffec9e8
recursive spill
dqhl76 Oct 21, 2025
542fb5a
make build pass
dqhl76 Oct 22, 2025
7bdc8c9
partition stream need finalize
dqhl76 Oct 22, 2025
1f8e230
refine event, make it more clearly to understand
dqhl76 Oct 22, 2025
d24f45d
fix: reset flag during new round begin
dqhl76 Oct 22, 2025
82efd60
debug
dqhl76 Oct 22, 2025
dfa47ca
debug
dqhl76 Oct 22, 2025
588e388
make lint
dqhl76 Oct 22, 2025
e936f9f
fix
dqhl76 Oct 22, 2025
0c45ea6
chore: add random_spill_percentage
dqhl76 Oct 22, 2025
2dd0f91
fix: spill flag not reset
dqhl76 Oct 22, 2025
7531bee
used for debug
dqhl76 Oct 22, 2025
b9f6da4
fix: skip if output datablock is empty (caused downstream processor m…
dqhl76 Oct 23, 2025
02a6c59
fix: add blocking layer
dqhl76 Oct 23, 2025
f418988
disable random_spill_percentage
dqhl76 Oct 23, 2025
9e1c7c3
fix: try fix cluster aggregate hang
dqhl76 Oct 25, 2025
9262694
fix: stackoverflow in ci
dqhl76 Oct 26, 2025
6f8a5de
improve performance
dqhl76 Oct 26, 2025
c814917
disable to ready merge
dqhl76 Oct 26, 2025
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
1 change: 1 addition & 0 deletions Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -399,6 +399,7 @@ opendal = { version = "0.53.2", features = [
"layers-fastrace",
"layers-prometheus-client",
"layers-async-backtrace",
"layers-blocking",
"services-s3",
"services-fs",
"services-gcs",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -148,6 +148,8 @@ impl IPhysicalPlan for AggregateFinal {
.get_enable_experimental_aggregate_hashtable()?;
let max_spill_io_requests = builder.settings.get_max_spill_io_requests()?;
let max_restore_worker = builder.settings.get_max_aggregate_restore_worker()?;
let experiment_aggregate_final =
builder.settings.get_enable_experiment_aggregate_final()?;

let mut is_cluster_aggregate = false;
if ExchangeSource::check_physical_plan(&self.input) {
Expand Down Expand Up @@ -199,6 +201,8 @@ impl IPhysicalPlan for AggregateFinal {
params.clone(),
max_restore_worker,
after_group_parallel,
experiment_aggregate_final,
builder.ctx.clone(),
)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,6 +107,7 @@ impl SerializedPayload {
}
}

#[derive(Debug)]
pub struct BucketSpilledPayload {
pub bucket: isize,
pub location: String,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,151 @@
// Copyright 2021 Datafuse Labs
//
// Licensed 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.

use std::sync::Arc;

use databend_common_exception::Result;
use databend_common_pipeline_core::processors::InputPort;
use databend_common_pipeline_core::processors::OutputPort;
use databend_common_pipeline_core::processors::ProcessorPtr;
use databend_common_pipeline_core::Pipe;
use databend_common_pipeline_core::PipeItem;
use databend_common_pipeline_core::Pipeline;
use databend_common_pipeline_core::TransformPipeBuilder;
use databend_common_storage::DataOperator;
use parking_lot::Mutex;
use tokio::sync::Barrier;
use tokio::sync::Semaphore;

use crate::pipelines::processors::transforms::aggregator::new_final_aggregate::FinalAggregateSharedState;
use crate::pipelines::processors::transforms::aggregator::new_final_aggregate::FinalAggregateSpiller;
use crate::pipelines::processors::transforms::aggregator::new_final_aggregate::NewFinalAggregateTransform;
use crate::pipelines::processors::transforms::aggregator::new_final_aggregate::TransformPartitionBucketScatter;
use crate::pipelines::processors::transforms::aggregator::transform_partition_bucket::TransformPartitionBucket;
use crate::pipelines::processors::transforms::aggregator::AggregatorParams;
use crate::pipelines::processors::transforms::aggregator::TransformAggregateSpillReader;
use crate::pipelines::processors::transforms::aggregator::TransformFinalAggregate;
use crate::sessions::QueryContext;

fn build_partition_bucket_experimental(
pipeline: &mut Pipeline,
params: Arc<AggregatorParams>,
after_worker: usize,
ctx: Arc<QueryContext>,
) -> Result<()> {
let operator = DataOperator::instance().spill_operator();

// PartitionedPayload only accept power of two partitions
let mut output_num = after_worker.next_power_of_two();
const MAX_PARTITION_COUNT: usize = 128;
if output_num > MAX_PARTITION_COUNT {
output_num = MAX_PARTITION_COUNT;
}

let input_num = pipeline.output_len();
let scatter = TransformPartitionBucketScatter::create(input_num, output_num, params.clone())?;
let scatter_inputs = scatter.get_inputs();
let scatter_outputs = scatter.get_outputs();

pipeline.add_pipe(Pipe::create(
scatter_inputs.len(),
scatter_outputs.len(),
vec![PipeItem::create(
ProcessorPtr::create(Box::new(scatter)),
scatter_inputs,
scatter_outputs,
)],
));

let mut builder = TransformPipeBuilder::create();
let barrier = Arc::new(Barrier::new(output_num));
let shared_state = Arc::new(Mutex::new(FinalAggregateSharedState::new(output_num)));

for id in 0..output_num {
let spiller = FinalAggregateSpiller::try_create(ctx.clone(), operator.clone())?;
let input_port = InputPort::create();
let output_port = OutputPort::create();
let processor = NewFinalAggregateTransform::try_create(
input_port.clone(),
output_port.clone(),
id,
params.clone(),
output_num,
barrier.clone(),
shared_state.clone(),
spiller,
ctx.clone(),
)?;
builder.add_transform(input_port, output_port, ProcessorPtr::create(processor));
}

pipeline.add_pipe(builder.finalize());
pipeline.resize(after_worker, true)?;

Ok(())
}

fn build_partition_bucket_legacy(
pipeline: &mut Pipeline,
params: Arc<AggregatorParams>,
max_restore_worker: u64,
after_worker: usize,
) -> Result<()> {
let operator = DataOperator::instance().spill_operator();

let input_nums = pipeline.output_len();
let transform = TransformPartitionBucket::create(input_nums, params.clone())?;

let output = transform.get_output();
let inputs_port = transform.get_inputs();

pipeline.add_pipe(Pipe::create(inputs_port.len(), 1, vec![PipeItem::create(
ProcessorPtr::create(Box::new(transform)),
inputs_port,
vec![output],
)]));

pipeline.try_resize(std::cmp::min(input_nums, max_restore_worker as usize))?;
let semaphore = Arc::new(Semaphore::new(params.max_spill_io_requests));
pipeline.add_transform(|input, output| {
let operator = operator.clone();
TransformAggregateSpillReader::create(input, output, operator, semaphore.clone())
})?;
pipeline.add_transform(|input, output| {
Ok(ProcessorPtr::create(TransformFinalAggregate::try_create(
input,
output,
params.clone(),
)?))
})?;
pipeline.try_resize(after_worker)?;

Ok(())
}

/// Build partition bucket pipeline based on the experiment_aggregate_final flag.
/// Dispatches to either experimental or legacy implementation.
pub fn build_partition_bucket(
pipeline: &mut Pipeline,
params: Arc<AggregatorParams>,
max_restore_worker: u64,
after_worker: usize,
experiment_aggregate_final: bool,
ctx: Arc<QueryContext>,
) -> Result<()> {
if experiment_aggregate_final {
build_partition_bucket_experimental(pipeline, params, after_worker, ctx)
} else {
build_partition_bucket_legacy(pipeline, params, max_restore_worker, after_worker)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,18 +15,20 @@
mod aggregate_exchange_injector;
mod aggregate_meta;
mod aggregator_params;
mod new_transform_partition_bucket;
mod build_partition_bucket;
mod new_final_aggregate;
mod serde;
mod transform_aggregate_expand;
mod transform_aggregate_final;
mod transform_aggregate_partial;
mod transform_partition_bucket;
mod transform_single_key;
mod udaf_script;

pub use aggregate_exchange_injector::AggregateInjector;
pub use aggregate_meta::*;
pub use aggregator_params::AggregatorParams;
pub use new_transform_partition_bucket::build_partition_bucket;
pub use build_partition_bucket::build_partition_bucket;
pub use transform_aggregate_expand::TransformExpandGroupingSets;
pub use transform_aggregate_final::TransformFinalAggregate;
pub use transform_aggregate_partial::TransformPartialAggregate;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
// Copyright 2021 Datafuse Labs
//
// Licensed 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.

use databend_common_expression::DataBlock;

use crate::pipelines::processors::transforms::aggregator::AggregateMeta;

/// Split partitioned metadata evenly into DataBlock chunks.
pub fn split_partitioned_meta_into_datablocks(
bucket: isize,
data: Vec<AggregateMeta>,
outputs_len: usize,
) -> Vec<DataBlock> {
if outputs_len == 0 {
return vec![];
}

let total_len = data.len();
let base_chunk_size = total_len / outputs_len;
let remainder = total_len % outputs_len;

let mut result = Vec::with_capacity(outputs_len);
let mut data_iter = data.into_iter();

for index in 0..outputs_len {
let chunk_size = if index < remainder {
base_chunk_size + 1
} else {
base_chunk_size
};

let chunk: Vec<AggregateMeta> = data_iter.by_ref().take(chunk_size).collect();
result.push(DataBlock::empty_with_meta(
AggregateMeta::create_partitioned(bucket, chunk),
));
}

result
}
Loading