alamb commented on issue #5146:
URL: 
https://github.com/apache/arrow-datafusion/issues/5146#issuecomment-1414358668

   I agree this is a bug. Thank you for the reproducer @bseifertNatzka  -- I 
could reproduce it just with a projection directly on the scan (also this 
happens on master at the time of this writing)
   
   I think you can work around it by using a LogicalPlanBuilder::project like 
this
   
   ```shell
       let plan = LogicalPlanBuilder::scan(
           "some_memtable",
           Arc::new(DefaultTableSource::new(mem_table)),
           //Some(vec![1, 0, 2]), <---- Don't project here, 
           None,
       )?
       .project(vec![col("Year"), col("Category"), col("Metric")])? // <-- 
PROJECT here instead
       .build()?;
       let physical_plan = ctx.state().create_physical_plan(&plan).await?;
       let _input = datafusion::physical_plan::execute_stream(physical_plan, 
ctx.task_ctx())?;
   ```
   
   
   
   ```rust
   
   #[tokio::main(flavor = "multi_thread", worker_threads = 10)]
   async fn main() -> Result<()> {
       let csv =
           r###"
   Category,Year,Metric
   Electronics,2010,10
   Electronics,2011,10
   Electronics,2012,10
   Pharmacy,2010,20
   Pharmacy,2011,30
   Pharmacy,2012,40
   "###;
   
       let ctx = SessionContext::new();
   
       let batches: Vec<RecordBatch> = 
datafusion::arrow::csv::ReaderBuilder::new()
           .has_header(true)
           .build(Cursor::new(csv))
           .unwrap()
           .collect::<Result<Vec<_>, ArrowError>>()
           .unwrap();
   
       let schema = batches
           .get(0)
           .expect("There should be at least 1 record batch")
           .schema();
   
       let mem_table = Arc::new(MemTable::try_new(
           schema,
           vec![batches],
       )?);
   
       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(())
   }
   ```
   
   


-- 
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