Skip to content

PhysicalOptimizer rule 'ProjectionPushdown' failed, due to generate a different schema #9004

@jonahgao

Description

@jonahgao

Describe the bug

Found in PR #8840.

When no projection is pushed down to TableProvider, the physical optimization rule ProjectionPushdown may fail.

I think the root cause is in this function, when the source parameter is None.
https://github.com/apache/arrow-datafusion/blob/8a4bad46540598c6acdf432bde08c2a4c76c5039/datafusion/core/src/physical_optimizer/projection_pushdown.rs#L834-L837

To Reproduce

There may be some complex queries that can reproduce the problem.
But the simplest way is to disable projection pushdown during the logical optimization phase.

DataFusion CLI v35.0.0

❯ create table t(a int, b int);
0 rows in set. Query took 0.007 seconds.

❯ set datafusion.optimizer.max_passes=0;
0 rows in set. Query took 0.002 seconds.

❯ select a from t;
ProjectionPushdown
caused by
Internal error: PhysicalOptimizer rule 'ProjectionPushdown' failed, due to generate a different schema, 

original schema: Schema { fields: [Field { name: "a", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {} }, 

new schema: Schema { fields: [], metadata: {} }.

This was likely caused by a bug in DataFusion's code and we would welcome that you file an bug report in our issue tracker

Expected behavior

No response

Additional context

Not only MemoryTable, others like CsvTable also have the same problem.

Metadata

Metadata

Assignees

No one assigned

    Labels

    bugSomething isn't working

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions