Skip to content

Commit

Permalink
Use node-level local limit (#20)
Browse files Browse the repository at this point in the history
* Use node-level local limit

* serialize limit in shuffle writer

* Revert "Merge pull request #19 from coralogix/sc-5792"

This reverts commit 08140ef, reversing
changes made to a7f1384.

* add log

* make sure we don't forget limit for shuffle writer

* update accum correctly and try to break early

* Check local limit accumulator before polling for more data

* fix build

Co-authored-by: Martins Purins <martins.purins@coralogix.com>
  • Loading branch information
2 people authored and fsdvh committed Feb 2, 2023
1 parent b8f8983 commit 0c79855
Show file tree
Hide file tree
Showing 7 changed files with 1,970 additions and 12 deletions.
2 changes: 2 additions & 0 deletions ballista/core/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -56,9 +56,11 @@ datafusion-proto = "17.0.0"
futures = "0.3"
hashbrown = "0.13"

lazy_static = "1.4.0"
itertools = "0.10"
libloading = "0.7.3"
log = "0.4"
lru = "0.8.1"
object_store = "0.5.2"
once_cell = "1.9.0"

Expand Down
128 changes: 128 additions & 0 deletions ballista/core/proto/ballista.proto
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,134 @@ message UnresolvedShuffleExecNode {
uint32 output_partition_count = 4;
}

message FilterExecNode {
PhysicalPlanNode input = 1;
PhysicalExprNode expr = 2;
}

message FileGroup {
repeated PartitionedFile files = 1;
}

message ScanLimit {
// wrap into a message to make it optional
uint32 limit = 1;
}

message FileScanExecConf {
repeated FileGroup file_groups = 1;
datafusion.Schema schema = 2;
repeated uint32 projection = 4;
ScanLimit limit = 5;
Statistics statistics = 6;
repeated string table_partition_cols = 7;
string object_store_url = 8;
}

message ParquetScanExecNode {
FileScanExecConf base_conf = 1;
datafusion.LogicalExprNode pruning_predicate = 2;
}

message CsvScanExecNode {
FileScanExecConf base_conf = 1;
bool has_header = 2;
string delimiter = 3;
}

message AvroScanExecNode {
FileScanExecConf base_conf = 1;
}

enum PartitionMode {
COLLECT_LEFT = 0;
PARTITIONED = 1;
}

message HashJoinExecNode {
PhysicalPlanNode left = 1;
PhysicalPlanNode right = 2;
repeated JoinOn on = 3;
datafusion.JoinType join_type = 4;
PartitionMode partition_mode = 6;
bool null_equals_null = 7;
JoinFilter filter = 8;
}

message UnionExecNode {
repeated PhysicalPlanNode inputs = 1;
}

message ExplainExecNode {
datafusion.Schema schema = 1;
repeated datafusion.StringifiedPlan stringified_plans = 2;
bool verbose = 3;
}

message CrossJoinExecNode {
PhysicalPlanNode left = 1;
PhysicalPlanNode right = 2;
}

message PhysicalColumn {
string name = 1;
uint32 index = 2;
}

message JoinOn {
PhysicalColumn left = 1;
PhysicalColumn right = 2;
}

message EmptyExecNode {
bool produce_one_row = 1;
datafusion.Schema schema = 2;
}

message ProjectionExecNode {
PhysicalPlanNode input = 1;
repeated PhysicalExprNode expr = 2;
repeated string expr_name = 3;
}

enum AggregateMode {
PARTIAL = 0;
FINAL = 1;
FINAL_PARTITIONED = 2;
}

message WindowAggExecNode {
PhysicalPlanNode input = 1;
repeated PhysicalExprNode window_expr = 2;
repeated string window_expr_name = 3;
datafusion.Schema input_schema = 4;
}

message AggregateExecNode {
repeated PhysicalExprNode group_expr = 1;
repeated PhysicalExprNode aggr_expr = 2;
AggregateMode mode = 3;
PhysicalPlanNode input = 4;
repeated string group_expr_name = 5;
repeated string aggr_expr_name = 6;
// we need the input schema to the partial aggregate to pass to the final aggregate
datafusion.Schema input_schema = 7;
repeated PhysicalExprNode null_expr = 8;
repeated bool groups = 9;
}

message ShuffleWriterExecNode {
//TODO it seems redundant to provide job and stage id here since we also have them
// in the TaskDefinition that wraps this plan
string job_id = 1;
uint32 stage_id = 2;
PhysicalPlanNode input = 3;
PhysicalHashRepartition output_partitioning = 4;
oneof optional_limit {
uint64 limit = 6;
}
}

message ShuffleReaderExecNode {
repeated ShuffleReaderPartition partition = 1;
datafusion.Schema schema = 2;
Expand Down
84 changes: 81 additions & 3 deletions ballista/core/src/execution_plans/shuffle_writer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,30 @@ use datafusion::arrow::error::ArrowError;
use datafusion::execution::context::TaskContext;
use datafusion::physical_plan::repartition::BatchPartitioner;
use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
use lazy_static::lazy_static;
use log::{debug, info};
use lru::LruCache;
use parking_lot::Mutex;
use std::num::NonZeroUsize;
use std::sync::atomic::{AtomicUsize, Ordering};

lazy_static! {
static ref LIMIT_ACCUMULATORS: Mutex<LruCache<(String, usize), Arc<AtomicUsize>>> =
Mutex::new(LruCache::new(NonZeroUsize::new(40).unwrap()));
}

fn get_limit_accumulator(job_id: &str, stage: usize) -> Arc<AtomicUsize> {
let mut guard = LIMIT_ACCUMULATORS.lock();

if let Some(accumulator) = guard.get(&(job_id.to_owned(), stage)) {
accumulator.clone()
} else {
let accumulator = Arc::new(AtomicUsize::new(0));
guard.push((job_id.to_owned(), stage), accumulator.clone());

accumulator
}
}

/// ShuffleWriterExec represents a section of a query plan that has consistent partitioning and
/// can be executed as one unit with each partition being executed in parallel. The output of each
Expand All @@ -75,6 +98,8 @@ pub struct ShuffleWriterExec {
shuffle_output_partitioning: Option<Partitioning>,
/// Execution metrics
metrics: ExecutionPlanMetricsSet,
/// Maximum number of rows to return
limit: Option<usize>,
}

#[derive(Debug, Clone)]
Expand Down Expand Up @@ -121,6 +146,26 @@ impl ShuffleWriterExec {
work_dir,
shuffle_output_partitioning,
metrics: ExecutionPlanMetricsSet::new(),
limit: None,
})
}

pub fn try_new_with_limit(
job_id: String,
stage_id: usize,
plan: Arc<dyn ExecutionPlan>,
work_dir: String,
shuffle_output_partitioning: Option<Partitioning>,
limit: Option<usize>,
) -> Result<Self> {
Ok(Self {
job_id,
stage_id,
plan,
work_dir,
shuffle_output_partitioning,
metrics: ExecutionPlanMetricsSet::new(),
limit,
})
}

Expand All @@ -139,6 +184,10 @@ impl ShuffleWriterExec {
self.shuffle_output_partitioning.as_ref()
}

pub fn limit(&self) -> Option<usize> {
self.limit
}

pub fn execute_shuffle_write(
&self,
input_partition: usize,
Expand All @@ -152,6 +201,10 @@ impl ShuffleWriterExec {
let output_partitioning = self.shuffle_output_partitioning.clone();
let plan = self.plan.clone();

let limit_and_accumulator = self
.limit
.map(|l| (l, get_limit_accumulator(&self.job_id, self.stage_id)));

async move {
let now = Instant::now();
let mut stream = plan.execute(input_partition, context)?;
Expand All @@ -170,6 +223,7 @@ impl ShuffleWriterExec {
&mut stream,
path,
&write_metrics.write_time,
limit_and_accumulator,
)
.await
.map_err(|e| DataFusionError::Execution(format!("{e:?}")))?;
Expand Down Expand Up @@ -211,10 +265,26 @@ impl ShuffleWriterExec {
write_metrics.repart_time.clone(),
)?;

while let Some(result) = stream.next().await {
while let Some(result) = {
let poll_more = limit_and_accumulator.as_ref().map_or(
true,
|(limit, accum)| {
let total_rows = accum.load(Ordering::SeqCst);
total_rows < *limit
},
);

if poll_more {
stream.next().await
} else {
None
}
} {
let input_batch = result?;

write_metrics.input_rows.add(input_batch.num_rows());
let num_rows = input_batch.num_rows();

write_metrics.input_rows.add(num_rows);

partitioner.partition(
input_batch,
Expand Down Expand Up @@ -252,6 +322,13 @@ impl ShuffleWriterExec {
Ok(())
},
)?;

if let Some((limit, accum)) = limit_and_accumulator.as_ref() {
let total_rows = accum.fetch_add(num_rows, Ordering::SeqCst);
if total_rows > *limit {
break;
}
}
}

let mut part_locs = vec![];
Expand Down Expand Up @@ -320,12 +397,13 @@ impl ExecutionPlan for ShuffleWriterExec {
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(ShuffleWriterExec::try_new(
Ok(Arc::new(ShuffleWriterExec::try_new_with_limit(
self.job_id.clone(),
self.stage_id,
children[0].clone(),
self.work_dir.clone(),
self.shuffle_output_partitioning.clone(),
self.limit,
)?))
}

Expand Down
Loading

0 comments on commit 0c79855

Please sign in to comment.