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

feat: dist sql analyze #1260

Merged
merged 20 commits into from
Dec 28, 2023
Merged
Show file tree
Hide file tree
Changes from 10 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
49 changes: 31 additions & 18 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 1 addition & 1 deletion Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ bytes = "1"
bytes_ext = { path = "components/bytes_ext" }
catalog = { path = "catalog" }
catalog_impls = { path = "catalog_impls" }
ceresdbproto = "1.0.22"
ceresdbproto = { git = "https://github.com/baojinri/ceresdbproto.git", rev = "9b05108ea41b323c7ef7e54bb570b59cca5e76ed" }
codec = { path = "components/codec" }
chrono = "0.4"
clap = "3.0"
Expand Down
28 changes: 28 additions & 0 deletions components/trace_metric/src/collector.rs
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,34 @@ impl CollectorVisitor for FormatCollectorVisitor {
}
}

/// A collector for metrics of remote plan.
baojinri marked this conversation as resolved.
Show resolved Hide resolved
///
/// It can be cloned and shared among threads.
#[derive(Clone, Debug, Default)]
pub struct RemoteMetricsCollector {
name: String,
metrics: Arc<Mutex<String>>,
}

impl RemoteMetricsCollector {
pub fn new(name: String) -> Self {
Self {
name,
metrics: Arc::new(Mutex::new(String::new())),
}
}

pub fn collect(&self, metric: String) {
let mut metrics = self.metrics.lock().unwrap();
metrics.push_str(metric.as_str());
}

pub fn display(&self) -> String {
let metrics = self.metrics.lock().unwrap();
format!("{}:\n{}", self.name, metrics)
}
}

#[cfg(test)]
mod tests {
use std::time::Duration;
Expand Down
2 changes: 2 additions & 0 deletions df_engine_extensions/src/dist_sql_query/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ use datafusion::{
use futures::future::BoxFuture;
use generic_error::BoxError;
use table_engine::{predicate::PredicateRef, remote::model::TableIdentifier, table::TableRef};
use trace_metric::collector::RemoteMetricsCollector;

pub mod codec;
pub mod physical_plan;
Expand All @@ -38,6 +39,7 @@ pub trait RemotePhysicalPlanExecutor: fmt::Debug + Send + Sync + 'static {
table: TableIdentifier,
task_context: &TaskContext,
plan: Arc<dyn ExecutionPlan>,
remote_metrics_collector: RemoteMetricsCollector,
) -> DfResult<BoxFuture<'static, DfResult<SendableRecordBatchStream>>>;
}

Expand Down
54 changes: 48 additions & 6 deletions df_engine_extensions/src/dist_sql_query/physical_plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@

use std::{
any::Any,
collections::HashMap,
fmt,
pin::Pin,
sync::Arc,
Expand Down Expand Up @@ -43,7 +44,10 @@ use datafusion::{
};
use futures::{future::BoxFuture, FutureExt, Stream, StreamExt};
use table_engine::{remote::model::TableIdentifier, table::ReadRequest};
use trace_metric::{collector::FormatCollectorVisitor, MetricsCollector, TraceMetricWhenDrop};
use trace_metric::{
collector::{FormatCollectorVisitor, RemoteMetricsCollector},
MetricsCollector, TraceMetricWhenDrop,
};

use crate::dist_sql_query::{RemotePhysicalPlanExecutor, TableScanContext};

Expand Down Expand Up @@ -147,31 +151,45 @@ pub(crate) struct ResolvedPartitionedScan {
pub remote_exec_ctx: Arc<RemoteExecContext>,
pub pushdown_continue: bool,
pub metrics_collector: MetricsCollector,
pub remote_metrics_collector: HashMap<String, RemoteMetricsCollector>,
baojinri marked this conversation as resolved.
Show resolved Hide resolved
pub is_analyze: bool,
}

impl ResolvedPartitionedScan {
pub fn new(
remote_executor: Arc<dyn RemotePhysicalPlanExecutor>,
sut_table_plan_ctxs: Vec<SubTablePlanContext>,
metrics_collector: MetricsCollector,
remote_metrics_collector: HashMap<String, RemoteMetricsCollector>,
is_analyze: bool,
) -> Self {
let remote_exec_ctx = Arc::new(RemoteExecContext {
executor: remote_executor,
plan_ctxs: sut_table_plan_ctxs,
});

Self::new_with_details(remote_exec_ctx, true, metrics_collector)
Self::new_with_details(
remote_exec_ctx,
true,
metrics_collector,
remote_metrics_collector,
is_analyze,
)
}

pub fn new_with_details(
remote_exec_ctx: Arc<RemoteExecContext>,
pushdown_continue: bool,
metrics_collector: MetricsCollector,
remote_metrics_collector: HashMap<String, RemoteMetricsCollector>,
is_analyze: bool,
) -> Self {
Self {
remote_exec_ctx,
pushdown_continue,
metrics_collector,
remote_metrics_collector,
is_analyze,
}
}

Expand All @@ -180,6 +198,8 @@ impl ResolvedPartitionedScan {
remote_exec_ctx: self.remote_exec_ctx.clone(),
pushdown_continue: false,
metrics_collector: self.metrics_collector.clone(),
remote_metrics_collector: self.remote_metrics_collector.clone(),
is_analyze: self.is_analyze,
})
}

Expand Down Expand Up @@ -227,6 +247,8 @@ impl ResolvedPartitionedScan {
remote_exec_ctx,
can_push_down_more,
self.metrics_collector.clone(),
self.remote_metrics_collector.clone(),
self.is_analyze,
);

Ok(Arc::new(plan))
Expand Down Expand Up @@ -295,6 +317,12 @@ impl ExecutionPlan for ResolvedPartitionedScan {
}

fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
// If this is a analyze plan, we should not collect metrics of children
// which have been send to remote, So we just return empty children.
if self.is_analyze {
return vec![];
}

self.remote_exec_ctx
.plan_ctxs
.iter()
Expand Down Expand Up @@ -328,12 +356,15 @@ impl ExecutionPlan for ResolvedPartitionedScan {
plan,
metrics_collector,
} = &self.remote_exec_ctx.plan_ctxs[partition];
let remote_metrics_collector = self.remote_metrics_collector.get(&sub_table.table).unwrap();

// Send plan for remote execution.
let stream_future =
self.remote_exec_ctx
.executor
.execute(sub_table.clone(), &context, plan.clone())?;
let stream_future = self.remote_exec_ctx.executor.execute(
sub_table.clone(),
&context,
plan.clone(),
remote_metrics_collector.clone(),
)?;
let record_stream =
PartitionedScanStream::new(stream_future, plan.schema(), metrics_collector.clone());

Expand All @@ -347,6 +378,17 @@ impl ExecutionPlan for ResolvedPartitionedScan {
fn metrics(&self) -> Option<MetricsSet> {
let mut metric_set = MetricsSet::new();

for remote_metrics_collector in self.remote_metrics_collector.values() {
let metrics_desc = remote_metrics_collector.display();
metric_set.push(Arc::new(Metric::new(
MetricValue::Count {
name: format!("\n{metrics_desc}").into(),
count: Count::new(),
},
None,
)));
}

let mut format_visitor = FormatCollectorVisitor::default();
self.metrics_collector.visit(&mut format_visitor);
let metrics_desc = format_visitor.into_string();
Expand Down
Loading