Skip to content

Commit

Permalink
restore metrics (to fix tests)
Browse files Browse the repository at this point in the history
  • Loading branch information
matthewgapp committed Sep 19, 2023
1 parent 5bbc5aa commit 815c72f
Show file tree
Hide file tree
Showing 3 changed files with 9 additions and 27 deletions.
16 changes: 0 additions & 16 deletions datafusion/execution/src/task.rs
Original file line number Diff line number Diff line change
Expand Up @@ -65,8 +65,6 @@ pub struct TaskContext {
window_functions: HashMap<String, Arc<WindowUDF>>,
/// Runtime environment associated with this task context
runtime: Arc<RuntimeEnv>,
// TODO: to remove, this is only for testing
active_partition: Mutex<Option<usize>>,
/// Registered relation handlers
relation_handlers: Mutex<HashMap<String, RelationHandler>>,
}
Expand All @@ -86,7 +84,6 @@ impl Default for TaskContext {
window_functions: HashMap::new(),
runtime: Arc::new(runtime),
relation_handlers: Mutex::new(HashMap::new()),
active_partition: Mutex::new(None),
}
}
}
Expand Down Expand Up @@ -115,19 +112,6 @@ impl TaskContext {
window_functions,
runtime,
relation_handlers: Mutex::new(HashMap::new()),
active_partition: Mutex::new(None),
}
}

pub fn set_and_increment_partition(&self) -> usize {
let mut partition_guard = self.active_partition.lock();
if let Some(ref mut partition) = &mut *partition_guard {
let prev = *partition;
*partition += 1;
prev
} else {
*partition_guard = Some(1);
0
}
}

Expand Down
11 changes: 5 additions & 6 deletions datafusion/physical-plan/src/filter.rs
Original file line number Diff line number Diff line change
Expand Up @@ -185,12 +185,12 @@ impl ExecutionPlan for FilterExec {
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
trace!("Start FilterExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id());
// let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
Ok(Box::pin(FilterExecStream {
schema: self.input.schema(),
predicate: self.predicate.clone(),
input: self.input.execute(partition, context)?,
// baseline_metrics,
baseline_metrics,
}))
}

Expand Down Expand Up @@ -296,8 +296,8 @@ struct FilterExecStream {
predicate: Arc<dyn PhysicalExpr>,
/// The input partition to filter.
input: SendableRecordBatchStream,
// /// runtime metrics recording
// baseline_metrics: BaselineMetrics,
/// runtime metrics recording
baseline_metrics: BaselineMetrics,
}

pub(crate) fn batch_filter(
Expand Down Expand Up @@ -347,8 +347,7 @@ impl Stream for FilterExecStream {
}
}
}
// self.baseline_metrics.record_poll(poll)
poll
self.baseline_metrics.record_poll(poll)
}

fn size_hint(&self) -> (usize, Option<usize>) {
Expand Down
9 changes: 4 additions & 5 deletions datafusion/physical-plan/src/projection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -324,7 +324,7 @@ impl ExecutionPlan for ProjectionExec {
schema: self.schema.clone(),
expr: self.expr.iter().map(|x| x.0.clone()).collect(),
input: self.input.execute(partition, context)?,
// baseline_metrics: BaselineMetrics::new(&self.metrics, partition),
baseline_metrics: BaselineMetrics::new(&self.metrics, partition),
}))
}

Expand Down Expand Up @@ -450,7 +450,7 @@ fn stats_projection(
impl ProjectionStream {
fn batch_project(&self, batch: &RecordBatch) -> Result<RecordBatch> {
// records time on drop
// let _timer = self.baseline_metrics.elapsed_compute().timer();
let _timer = self.baseline_metrics.elapsed_compute().timer();
let arrays = self
.expr
.iter()
Expand All @@ -474,7 +474,7 @@ struct ProjectionStream {
schema: SchemaRef,
expr: Vec<Arc<dyn PhysicalExpr>>,
input: SendableRecordBatchStream,
// baseline_metrics: BaselineMetrics,
baseline_metrics: BaselineMetrics,
}

impl Stream for ProjectionStream {
Expand All @@ -489,8 +489,7 @@ impl Stream for ProjectionStream {
other => other,
});

// self.baseline_metrics.record_poll(poll)
poll
self.baseline_metrics.record_poll(poll)
}

fn size_hint(&self) -> (usize, Option<usize>) {
Expand Down

0 comments on commit 815c72f

Please sign in to comment.