bseifertNatzka opened a new issue, #5146:
URL: https://github.com/apache/arrow-datafusion/issues/5146

   **Describe the bug**
   Projections for re-ordering the columns of schemas don't work anymore in 
`LogicalPlanBuilder::scan` (since DF15), because the schema changes 
unexpectedly during Optimizer rule 'push_down_projection'.
   
   **To Reproduce**
   Steps to reproduce the behavior:
   Minimal example:
   ```rust
   use std::io::Cursor;
   use std::sync::Arc;
   
   use futures::TryStreamExt;
   use datafusion::{
       arrow::
           record_batch::RecordBatch,
       datasource::{DefaultTableSource, MemTable},
       logical_expr::LogicalPlanBuilder,
       physical_plan::{memory::MemoryStream, SendableRecordBatchStream},
       prelude::SessionContext,
   };
   
   use itertools::Itertools;
   
   fn main() {
   }
   
   #[tokio::test]
   async fn with_projection() -> Result<(), anyhow::Error> {
       let input = create_record_batches_from_csv(
           r###"
   Category,Year,Metric
   Electronics,2010,10
   Electronics,2011,10
   Electronics,2012,10
   Pharmacy,2010,20
   Pharmacy,2011,30
   Pharmacy,2012,40
   "###,
       )
       .await?;
       let ctx = SessionContext::new();
       let mem_table = Arc::new(MemTable::try_new(
           input.schema(),
           vec![input.try_collect().await?],
       )?);
       let plan = LogicalPlanBuilder::scan(
           "some_memtable",
           Arc::new(DefaultTableSource::new(mem_table)),
           Some(vec![1, 0, 2]),
       )?
       .build()?;
       let physical_plan = ctx.state().create_physical_plan(&plan).await?;
       let _input = datafusion::physical_plan::execute_stream(physical_plan, 
ctx.task_ctx())?;
       
       Ok(())
   }
   
   
   pub async fn create_record_batches_from_csv(
       csv: &str,
   ) -> Result<SendableRecordBatchStream, anyhow::Error> {
       let csv_reader: Vec<RecordBatch> = 
datafusion::arrow::csv::ReaderBuilder::new()
           .has_header(true)
           .build(Cursor::new(csv))?
           .try_collect()?;
       let schema = csv_reader
           .get(0)
           .expect("There should be at least 1 record batch")
           .schema();
   
       Ok(Box::pin(MemoryStream::try_new(csv_reader, schema, None)?))
   }
   ```
   with Cargo.toml
   ```
   [package]
   name = "df-bug"
   version = "0.1.0"
   edition = "2021"
   
   # See more keys and their definitions at 
https://doc.rust-lang.org/cargo/reference/manifest.html
   
   [dependencies]
   datafusion = "17"
   arrow = "31"      # IMPORTANT: keep in sync with the version that 
`datafusion`uses
   
   anyhow = "1.0"
   futures = "0.3"
   itertools = "0.10"
   tokio = "1"
   tokio-retry = "0.3"
   tokio-stream = "0.1"
   tokio-util = "0.7"
   ```
   yields the error
   ```
   Error: Internal error: Optimizer rule 'push_down_projection' failed, due to 
generate a different schema, original schema: 
   DFSchema { fields: 
   [
   DFField { qualifier: Some("some_memtable"), field: Field { name: "Year", 
data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: 
{} } }, 
   DFField { qualifier: Some("some_memtable"), field: Field { name: "Category", 
data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: 
{} } }, 
   DFField { qualifier: Some("some_memtable"), field: Field { name: "Metric", 
data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: 
{} } }
   ], metadata: {} }, 
   
   new schema: DFSchema { fields:
    [
   DFField { qualifier: Some("some_memtable"), field: Field { name: "Category", 
data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: 
{} } }, 
   DFField { qualifier: Some("some_memtable"), field: Field { name: "Year", 
data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: 
{} } },
    DFField { qualifier: Some("some_memtable"), field: Field { name: "Metric", 
data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: 
{} } }
   ], 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**
   In DF14 this code did not yield an error (and indeed should not). Instead 
the output schema was changed as expected.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to