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

Take the top level schema into account when creating UnionExec #4753

Merged
merged 3 commits into from
Dec 30, 2022
Merged
Show file tree
Hide file tree
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
11 changes: 9 additions & 2 deletions datafusion/core/src/physical_plan/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -773,12 +773,19 @@ impl DefaultPhysicalPlanner {
)?;
Ok(Arc::new(FilterExec::try_new(runtime_expr, physical_input)?))
}
LogicalPlan::Union(Union { inputs, .. }) => {
LogicalPlan::Union(Union { inputs, schema }) => {
let physical_plans = futures::stream::iter(inputs)
.then(|lp| self.create_initial_plan(lp, session_state))
.try_collect::<Vec<_>>()
.await?;
Ok(Arc::new(UnionExec::new(physical_plans)))
if schema.fields().len() < physical_plans[0].schema().fields().len() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder what is the reason to not always run UnionExec::try_new_with_schema?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The original implementation ignores the top level schema, but I don't know why. (Maybe @gandronchik knows more about it). So I try to minimize the scope of this change to avoid introducing new bugs.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good -- maybe I'll try to simply the code once we have merged this PR

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, after double thinking, I am uncertain whether it is correct to ignore the top level schema when generating physical plans. It would lose the information about type coercion and metadata.

// `schema` could be a subset of the child schema. For example
// for query "select count(*) from (select a from t union all select a from t)"
// `schema` is empty but child schema contains one field `a`.
Ok(Arc::new(UnionExec::try_new_with_schema(physical_plans, schema.clone())?))
} else {
Ok(Arc::new(UnionExec::new(physical_plans)))
}
}
LogicalPlan::Repartition(Repartition {
input,
Expand Down
33 changes: 33 additions & 0 deletions datafusion/core/src/physical_plan/union.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ use arrow::{
datatypes::{Field, Schema, SchemaRef},
record_batch::RecordBatch,
};
use datafusion_common::{DFSchemaRef, DataFusionError};
use futures::{Stream, StreamExt};
use itertools::Itertools;
use log::debug;
Expand Down Expand Up @@ -63,6 +64,38 @@ pub struct UnionExec {
}

impl UnionExec {
/// Create a new UnionExec with specified schema.
/// The `schema` should always be a subset of the schema of `inputs`,
/// otherwise, an error will be returned.
pub fn try_new_with_schema(
inputs: Vec<Arc<dyn ExecutionPlan>>,
schema: DFSchemaRef,
) -> Result<Self> {
let mut exec = Self::new(inputs);
let exec_schema = exec.schema();
let fields = schema
.fields()
.iter()
.map(|dff| {
exec_schema
.field_with_name(dff.name())
.cloned()
.map_err(|_| {
DataFusionError::Internal(format!(
"Cannot find the field {:?} in child schema",
dff.name()
))
})
})
.collect::<Result<Vec<Field>>>()?;
let schema = Arc::new(Schema::new_with_metadata(
fields,
exec.schema().metadata().clone(),
));
exec.schema = schema;
Ok(exec)
}

/// Create a new UnionExec
pub fn new(inputs: Vec<Arc<dyn ExecutionPlan>>) -> Self {
let fields: Vec<Field> = (0..inputs[0].schema().fields().len())
Expand Down
17 changes: 17 additions & 0 deletions datafusion/core/tests/sql/union.rs
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,23 @@ async fn union_all_with_aggregate() -> Result<()> {
Ok(())
}

#[tokio::test]
async fn union_all_with_count() -> Result<()> {
let ctx = SessionContext::new();
execute_to_batches(&ctx, "CREATE table t as SELECT 1 as a").await;
let sql = "SELECT COUNT(*) FROM (SELECT a from t UNION ALL SELECT a from t)";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍 agrees with postgres:

postgres=# CREATE table t as SELECT 1 as a;
SELECT 1
postgres=# SELECT COUNT(*) FROM (SELECT a from t UNION ALL SELECT a from t);
ERROR:  subquery in FROM must have an alias
LINE 1: SELECT COUNT(*) FROM (SELECT a from t UNION ALL SELECT a fro...
                             ^
HINT:  For example, FROM (SELECT ...) [AS] foo.
postgres=# SELECT COUNT(*) FROM (SELECT a from t UNION ALL SELECT a from t) as sq;
 count 
-------
     2
(1 row)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BTW, I'm not sure if it was a bug, but the PushDownProjection doesn't push down the empty projection to the child plans of Union in this example. Maybe @jackwener knows more about it.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BTW, I'm not sure if it was a bug, but the PushDownProjection doesn't push down the empty projection to the child plans of Union in this example. Maybe @jackwener knows more about it.

we need an issue to track this problem, I think there are some different ways of dealing with this problem.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please file an issue @jackwener ?

let actual = execute_to_batches(&ctx, sql).await;
let expected = vec![
"+-----------------+",
"| COUNT(UInt8(1)) |",
"+-----------------+",
"| 2 |",
"+-----------------+",
];
assert_batches_eq!(expected, &actual);
Ok(())
}

#[tokio::test]
async fn union_schemas() -> Result<()> {
let ctx =
Expand Down