albel727 opened a new issue, #4674: URL: https://github.com/apache/arrow-datafusion/issues/4674
A `JOIN USING` SQL statement with multiple joined-over columns produces a Dataframe, which contains only the first of the joined-over columns. ## To Reproduce Assume the following two tables. ``` +------+------+------+------+ | key1 | key2 | key3 | data | +------+------+------+------+ | 1 | 1 | 1 | 10 | +------+------+------+------+ +------+------+------+------+ | key1 | key2 | key3 | data | +------+------+------+------+ | 1 | 1 | 1 | 20 | +------+------+------+------+ ``` Let's join over them: ```sql SELECT * FROM tbl1 JOIN tbl2 USING (key1, key2, key3) ``` ## Expected result ``` +------+------+------+------+------+ | key1 | key2 | key3 | data | data | +------+------+------+------+------+ | 1 | 1 | 1 | 10 | 20 | +------+------+------+------+------+ ``` ## Actual result ``` +------+------+------+ | key1 | data | data | +------+------+------+ | 1 | 10 | 20 | +------+------+------+ ``` ## Example code ```rust use std::sync::Arc; use datafusion::arrow::array::Int32Array; use datafusion::arrow::datatypes::{DataType, Field, Schema}; use datafusion::arrow::record_batch::RecordBatch; use datafusion::error::Result; use datafusion::from_slice::FromSlice; use datafusion::prelude::*; #[tokio::main] async fn main() -> Result<()> { let ctx = SessionContext::new(); let schema = Arc::new(Schema::new(vec![ Field::new("key1", DataType::Int32, false), Field::new("key2", DataType::Int32, false), Field::new("key3", DataType::Int32, false), Field::new("data", DataType::Int32, false), ])); ctx.register_batch("tbl1", RecordBatch::try_new( schema.clone(), vec![ Arc::new(Int32Array::from_slice([1])), Arc::new(Int32Array::from_slice([1])), Arc::new(Int32Array::from_slice([1])), Arc::new(Int32Array::from_slice([10])), ], )?)?; ctx.register_batch("tbl2", RecordBatch::try_new( schema.clone(), vec![ Arc::new(Int32Array::from_slice([1])), Arc::new(Int32Array::from_slice([1])), Arc::new(Int32Array::from_slice([1])), Arc::new(Int32Array::from_slice([20])), ], )?)?; ctx.table("tbl1")?.show().await?; ctx.table("tbl2")?.show().await?; let df = ctx.sql("SELECT * FROM tbl1 JOIN tbl2 USING (key1, key2, key3)").await?; df.show().await?; // It can be worked around by mentioning the columns explicitly in the same SELECT. let df = ctx.sql("SELECT key2, key3, * FROM tbl1 JOIN tbl2 USING (key1, key2, key3)").await?; df.show().await?; // But an indirect access to a missing column fails. let df = ctx.sql("SELECT key2 FROM (SELECT * FROM tbl1 JOIN tbl2 USING (key1, key2, key3))").await?; df.show().await?; // Panics with // Error: SchemaError(FieldNotFound { // field: Column { relation: None, name: "key2" }, // valid_fields: Some([Column { relation: Some("tbl1"), name: "key1" }, // Column { relation: Some("tbl1"), name: "data" }, // Column { relation: Some("tbl2"), name: "data" }]) // }) 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: github-unsubscr...@arrow.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org