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

Projection pushdown removes unqualified column names even when they are used #617

Closed
alamb opened this issue Jun 24, 2021 · 7 comments · Fixed by #619
Closed

Projection pushdown removes unqualified column names even when they are used #617

alamb opened this issue Jun 24, 2021 · 7 comments · Fixed by #619
Labels
bug Something isn't working datafusion Changes in the datafusion crate

Comments

@alamb
Copy link
Contributor

alamb commented Jun 24, 2021

Describe the bug

Problem:

I have a (user defined node) that looks like

ExtensionNode (expr uses col1, col2, col3) --> outputs new_col
  TableScan t1 of (col1, col2, col3)

Prior to #55 this worked

After that PR the projection pushdown logic decides that since the col1, col2 and col3 references don't have the table qualifier t1 on them, they are removed and the optimized plan looks like

ExtensionNode (expr uses col1, col2, col3) --> outputs new_col
  TableScan t1 of (col1)

Which then has issues because it expects col2 and col3 to be present but they have been "optimized" out

Expected behavior
The table scan should include col2 and col3 (they should not be optimized out), in addition to col1

@alamb alamb added bug Something isn't working datafusion Changes in the datafusion crate labels Jun 24, 2021
@alamb
Copy link
Contributor Author

alamb commented Jun 24, 2021

FYI @houqp -- I believe I have found the problem, but now I need to sort out one other failure before I can make a PR

@alamb
Copy link
Contributor Author

alamb commented Jun 24, 2021

🤔 I think this may be more complicated than I thought - I am not sure if I am creating the plans incorrectly or if projection pushdown is doing the wrong thing. I need to study this some more

@alamb
Copy link
Contributor Author

alamb commented Jun 24, 2021

So at least initially the answer appears to be "I should be creating fully qualified Exprs (with relation name)"

Here is the change I needed to make in IOx:

         let exprs = input
             .schema()
             .fields()
             .iter()
-            .map(|field| logical_plan::col(field.name()))
+            .map(|field| Expr::Column(field.qualified_column()))
             .collect::<Vec<_>>();

This was a non obvious failure more however, and I can probably figure out a better way to detect such errors. Anyhow, closing for now

@alamb alamb closed this as completed Jun 24, 2021
@houqp
Copy link
Member

houqp commented Jun 24, 2021

hmm... the projection push down should be able to handle unqualified columns without problem, i.e. the current design doesn't require clients to add qualifiers to all the fields. Do you have a reproducible example i can play with? I definitely didn't test the change with extension nodes.

@alamb
Copy link
Contributor Author

alamb commented Jun 24, 2021

@houqp I will get one

@alamb alamb reopened this Jun 24, 2021
@alamb
Copy link
Contributor Author

alamb commented Jun 25, 2021

Here is a reproducer (in projection_push_down.rs) -- it can also be found on https://github.com/alamb/arrow-datafusion/tree/alamb/repro_projection_pruning

    #[test]
    fn table_scan_projected_schema_non_qualified_relation() -> Result<()> {
        let table_scan = test_table_scan()?;
        let input_schema = table_scan.schema();
        assert_eq!(3, input_schema.fields().len());
        assert_fields_eq(&table_scan, vec!["a", "b", "c"]);

        // Build the LogicalPlan directly (don't use PlanBuilder), so
        // that the Column references are unqualified (e.g. their
        // relation is `None`). PlanBuilder resolves the expressions
        let expr = vec![col("a"), col("b")];
        let projected_fields = exprlist_to_fields(&expr, input_schema).unwrap();
        let projected_schema = DFSchema::new(projected_fields).unwrap();
        let plan = LogicalPlan::Projection {
            expr,
            input: Arc::new(table_scan),
            schema: Arc::new(projected_schema),
        };

        assert_fields_eq(&plan, vec!["a", "b"]);

        let expected = "Projection: #a, #b\
        \n  TableScan: test projection=Some([0, 1])";

        assert_optimized_plan_eq(&plan, expected);

        Ok(())
    }

It fails in the following way:

---- optimizer::projection_push_down::tests::table_scan_projected_schema_non_qualified_relation stdout ----
thread 'optimizer::projection_push_down::tests::table_scan_projected_schema_non_qualified_relation' panicked at 'assertion failed: `(left == right)`
  left: `"Projection: #a, #b\n  TableScan: test projection=Some([0])"`,
 right: `"Projection: #test.a, #test.b\n  TableScan: test projection=Some([0, 1])"`', datafusion/src/optimizer/projection_push_down.rs:745:9
note: run with `RUST_BACKTRACE=1` environment variable to display a backtrace

The problem is that the TableScan is only scanning one column, even though the Projection is using two columns.

I will put up a PR with code to fix the problem

@alamb
Copy link
Contributor Author

alamb commented Jun 25, 2021

#619

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working datafusion Changes in the datafusion crate
Projects
None yet
Development

Successfully merging a pull request may close this issue.

2 participants