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

Fix build on main due to logical conflict #6875

Merged
merged 1 commit into from
Jul 7, 2023
Merged
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
72 changes: 29 additions & 43 deletions datafusion/core/src/physical_plan/streaming.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,7 @@ use log::debug;

use crate::datasource::physical_plan::{OutputOrderingDisplay, ProjectSchemaDisplay};
use crate::physical_plan::stream::RecordBatchStreamAdapter;
use crate::physical_plan::{
DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream,
};
use crate::physical_plan::{ExecutionPlan, Partitioning, SendableRecordBatchStream};
use datafusion_execution::TaskContext;

use super::{DisplayAs, DisplayFormatType};
Expand Down Expand Up @@ -106,7 +104,34 @@ impl DisplayAs for StreamingTableExec {
) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
write!(f, "StreamingTableExec")
write!(
f,
"StreamingTableExec: partition_sizes={:?}",
self.partitions.len(),
)?;
if !self.projected_schema.fields().is_empty() {
write!(
f,
", projection={}",
ProjectSchemaDisplay(&self.projected_schema)
)?;
}
if self.infinite {
write!(f, ", infinite_source=true")?;
}

self.projected_output_ordering
.as_deref()
.map_or(Ok(()), |ordering| {
if !ordering.is_empty() {
write!(
f,
", output_ordering={}",
OutputOrderingDisplay(ordering)
)?;
}
Ok(())
})
}
}
}
Expand Down Expand Up @@ -164,45 +189,6 @@ impl ExecutionPlan for StreamingTableExec {
})
}

fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
write!(
f,
"StreamingTableExec: partition_sizes={:?}",
self.partitions.len(),
)?;
if !self.projected_schema.fields().is_empty() {
write!(
f,
", projection={}",
ProjectSchemaDisplay(&self.projected_schema)
)?;
}
if self.infinite {
write!(f, ", infinite_source=true")?;
}

self.projected_output_ordering
.as_deref()
.map_or(Ok(()), |ordering| {
if !ordering.is_empty() {
write!(
f,
", output_ordering={}",
OutputOrderingDisplay(ordering)
)?;
}
Ok(())
})
}
}
}

fn statistics(&self) -> Statistics {
Default::default()
}
Expand Down