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

   ### Describe the bug
   
   When inserting values into a table (mem table) that has a schema where each 
field is non-nullable and and then performing a window function with a 
partition by clause, the following runtime panic occurs. 
   
   ```text
   called `Result::unwrap()` on an `Err` value: 
ArrowError(InvalidArgumentError("batches[0] schema is different with argument 
schema.\n            batches[0] schema: Schema { fields: [Field { name: \"id\", 
data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: 
{} }, Field { name: \"name\", data_type: Utf8, nullable: true, dict_id: 0, 
dict_is_ordered: false, metadata: {} }], metadata: {} },\n            argument 
schema: Schema { fields: [Field { name: \"id\", data_type: Int32, nullable: 
false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: 
\"name\", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, 
metadata: {} }], metadata: {} }\n            "))
   ```
   
   ### To Reproduce
   
    To illustrate, @xhwhis created a minimal reproducible example below. Note 
that the `insert` query and the `partition` keyword in the window function are 
required to throw an error. 
   
   ```rust
   use std::sync::Arc;
   
   use datafusion::{
       arrow::{
           datatypes::{DataType, Field, Schema},
           util::pretty::print_batches,
       },
       datasource::MemTable,
       prelude::{SessionConfig, SessionContext},
   };
   
   #[tokio::main(flavor = "current_thread")]
   async fn main() {
       let config = SessionConfig::new()
           .with_create_default_catalog_and_schema(true)
           .with_information_schema(true);
   
       let ctx = SessionContext::with_config(config);
   
       ctx.register_table("source_table", Arc::new(create_mem_table()))
           .unwrap();
   
       let insert_table_query = r#"INSERT INTO source_table VALUES (1, 
'Alice'),(2, 'Bob'),(3, 'Charlie'),(4, 'David'), (5, 'Eve')"#;
       let _ = ctx
           .sql(insert_table_query)
           .await
           .unwrap()
           .collect()
           .await
           .unwrap();
   
       let create_table_query =
           r#"SELECT *, RANK() OVER (PARTITION BY id) AS row_num FROM 
source_table"#;
   
       let batches = ctx
           .sql(create_table_query)
           .await
           .unwrap()
           .collect()
           .await
           .unwrap();
   
       print_batches(&batches).unwrap();
   }
   
   fn create_mem_table() -> MemTable {
       let schema = Arc::new(Schema::new(vec![
           Field::new("id", DataType::Int32, false),
           Field::new("name", DataType::Utf8, false),
       ]));
   
       MemTable::try_new(schema, vec![vec![]]).unwrap()
   }
   ```
   
   ### Expected behavior
   
   No panic, the schema set at insert time should reflect that of the table it 
was inserted into
   
   ### Additional context
   
   _No response_


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