You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "alamb (via GitHub)" <gi...@apache.org> on 2023/02/06 13:58:32 UTC

[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #5106: Add unnest_column to DataFrame

alamb commented on code in PR #5106:
URL: https://github.com/apache/arrow-datafusion/pull/5106#discussion_r1097395782


##########
datafusion/core/src/physical_plan/unnest.rs:
##########
@@ -0,0 +1,305 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Defines the unnest column plan for unnesting values in a column that contains a list
+//! type, conceptually is like joining each row with all the values in the list column.
+use arrow::array::{
+    new_null_array, Array, ArrayAccessor, ArrayRef, FixedSizeListArray, LargeListArray,
+    ListArray,
+};
+use arrow::datatypes::{Schema, SchemaRef};
+use arrow::record_batch::RecordBatch;
+use async_trait::async_trait;
+use futures::Stream;
+use futures::StreamExt;
+use log::debug;
+use std::time::Instant;
+use std::{any::Any, sync::Arc};
+
+use crate::execution::context::TaskContext;
+use crate::physical_plan::{
+    coalesce_batches::concat_batches, expressions::Column, DisplayFormatType,
+    Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, PhysicalExpr,
+    PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+use crate::{
+    error::{DataFusionError, Result},
+    scalar::ScalarValue,
+};
+
+/// Unnest the given column by joining the row with each value in the nested type.
+#[derive(Debug)]
+pub struct UnnestExec {
+    /// Input execution plan
+    input: Arc<dyn ExecutionPlan>,
+    /// The schema once the unnest is applied
+    schema: SchemaRef,
+    /// The unnest column
+    column: Column,
+}
+
+impl UnnestExec {
+    /// Create a new [UnnestExec].
+    pub fn new(input: Arc<dyn ExecutionPlan>, column: Column, schema: SchemaRef) -> Self {
+        UnnestExec {
+            input,
+            schema,
+            column,
+        }
+    }
+}
+
+impl ExecutionPlan for UnnestExec {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+
+    fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
+        vec![self.input.clone()]
+    }
+
+    /// Specifies whether this plan generates an infinite stream of records.
+    /// If the plan does not support pipelining, but it its input(s) are
+    /// infinite, returns an error to indicate this.    
+    fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
+        Ok(children[0])
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> Result<Arc<dyn ExecutionPlan>> {
+        Ok(Arc::new(UnnestExec::new(
+            children[0].clone(),
+            self.column.clone(),
+            self.schema.clone(),
+        )))
+    }
+
+    fn required_input_distribution(&self) -> Vec<Distribution> {
+        vec![Distribution::UnspecifiedDistribution]
+    }
+
+    fn output_partitioning(&self) -> Partitioning {
+        self.input.output_partitioning()
+    }
+
+    fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
+        None
+    }
+
+    fn equivalence_properties(&self) -> EquivalenceProperties {
+        self.input.equivalence_properties()
+    }
+
+    fn execute(
+        &self,
+        partition: usize,
+        context: Arc<TaskContext>,
+    ) -> Result<SendableRecordBatchStream> {
+        let input = self.input.execute(partition, context)?;
+
+        Ok(Box::pin(UnnestStream {
+            input,
+            schema: self.schema.clone(),
+            column: self.column.clone(),
+            num_input_batches: 0,
+            num_input_rows: 0,
+            num_output_batches: 0,
+            num_output_rows: 0,
+            unnest_time: 0,
+        }))
+    }
+
+    fn fmt_as(
+        &self,
+        t: DisplayFormatType,
+        f: &mut std::fmt::Formatter,
+    ) -> std::fmt::Result {
+        match t {
+            DisplayFormatType::Default => {
+                write!(f, "UnnestExec")
+            }
+        }
+    }
+
+    fn statistics(&self) -> Statistics {
+        self.input.statistics()
+    }
+}
+
+/// A stream that issues [RecordBatch]es with unnested column data.
+struct UnnestStream {
+    /// Input stream
+    input: SendableRecordBatchStream,
+    /// Unnested schema
+    schema: Arc<Schema>,
+    /// The unnest column
+    column: Column,
+    /// number of input batches
+    num_input_batches: usize,
+    /// number of input rows
+    num_input_rows: usize,
+    /// number of batches produced
+    num_output_batches: usize,
+    /// number of rows produced
+    num_output_rows: usize,
+    /// total time for column unnesting
+    unnest_time: usize,

Review Comment:
   ```suggestion
       /// total time for column unnesting, in ms
       unnest_time: usize,
   ```



##########
datafusion/core/tests/dataframe.rs:
##########
@@ -500,6 +503,110 @@ async fn right_anti_filter_push_down() -> Result<()> {
     Ok(())
 }
 
+#[tokio::test]
+async fn unnest_columns() -> Result<()> {
+    const NUM_ROWS: usize = 4;
+    let df = table_with_nested_types(NUM_ROWS).await?;
+    let results = df.collect().await?;
+    let expected = vec![
+        r#"+----------+------------------------------------------------------------+--------------------+"#,
+        r#"| shape_id | points                                                     | tags               |"#,
+        r#"+----------+------------------------------------------------------------+--------------------+"#,
+        r#"| 1        | [{"x": -3, "y": -4}, {"x": -3, "y": 6}, {"x": 2, "y": -2}] | [tag1]             |"#,
+        r#"| 2        |                                                            | [tag1, tag2]       |"#,

Review Comment:
   👍  for testing with null



##########
datafusion/core/src/physical_plan/unnest.rs:
##########
@@ -0,0 +1,305 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Defines the unnest column plan for unnesting values in a column that contains a list
+//! type, conceptually is like joining each row with all the values in the list column.
+use arrow::array::{
+    new_null_array, Array, ArrayAccessor, ArrayRef, FixedSizeListArray, LargeListArray,
+    ListArray,
+};
+use arrow::datatypes::{Schema, SchemaRef};
+use arrow::record_batch::RecordBatch;
+use async_trait::async_trait;
+use futures::Stream;
+use futures::StreamExt;
+use log::debug;
+use std::time::Instant;
+use std::{any::Any, sync::Arc};
+
+use crate::execution::context::TaskContext;
+use crate::physical_plan::{
+    coalesce_batches::concat_batches, expressions::Column, DisplayFormatType,
+    Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, PhysicalExpr,
+    PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+use crate::{
+    error::{DataFusionError, Result},
+    scalar::ScalarValue,
+};
+
+/// Unnest the given column by joining the row with each value in the nested type.
+#[derive(Debug)]
+pub struct UnnestExec {
+    /// Input execution plan
+    input: Arc<dyn ExecutionPlan>,
+    /// The schema once the unnest is applied
+    schema: SchemaRef,
+    /// The unnest column
+    column: Column,
+}
+
+impl UnnestExec {
+    /// Create a new [UnnestExec].
+    pub fn new(input: Arc<dyn ExecutionPlan>, column: Column, schema: SchemaRef) -> Self {
+        UnnestExec {
+            input,
+            schema,
+            column,
+        }
+    }
+}
+
+impl ExecutionPlan for UnnestExec {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+
+    fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
+        vec![self.input.clone()]
+    }
+
+    /// Specifies whether this plan generates an infinite stream of records.
+    /// If the plan does not support pipelining, but it its input(s) are
+    /// infinite, returns an error to indicate this.    
+    fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
+        Ok(children[0])
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> Result<Arc<dyn ExecutionPlan>> {
+        Ok(Arc::new(UnnestExec::new(
+            children[0].clone(),
+            self.column.clone(),
+            self.schema.clone(),
+        )))
+    }
+
+    fn required_input_distribution(&self) -> Vec<Distribution> {
+        vec![Distribution::UnspecifiedDistribution]
+    }
+
+    fn output_partitioning(&self) -> Partitioning {
+        self.input.output_partitioning()
+    }
+
+    fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
+        None
+    }
+
+    fn equivalence_properties(&self) -> EquivalenceProperties {
+        self.input.equivalence_properties()
+    }
+
+    fn execute(
+        &self,
+        partition: usize,
+        context: Arc<TaskContext>,
+    ) -> Result<SendableRecordBatchStream> {
+        let input = self.input.execute(partition, context)?;
+
+        Ok(Box::pin(UnnestStream {
+            input,
+            schema: self.schema.clone(),
+            column: self.column.clone(),
+            num_input_batches: 0,
+            num_input_rows: 0,
+            num_output_batches: 0,
+            num_output_rows: 0,
+            unnest_time: 0,
+        }))
+    }
+
+    fn fmt_as(
+        &self,
+        t: DisplayFormatType,
+        f: &mut std::fmt::Formatter,
+    ) -> std::fmt::Result {
+        match t {
+            DisplayFormatType::Default => {
+                write!(f, "UnnestExec")
+            }
+        }
+    }
+
+    fn statistics(&self) -> Statistics {
+        self.input.statistics()
+    }
+}
+
+/// A stream that issues [RecordBatch]es with unnested column data.
+struct UnnestStream {
+    /// Input stream
+    input: SendableRecordBatchStream,
+    /// Unnested schema
+    schema: Arc<Schema>,
+    /// The unnest column
+    column: Column,
+    /// number of input batches
+    num_input_batches: usize,
+    /// number of input rows
+    num_input_rows: usize,
+    /// number of batches produced
+    num_output_batches: usize,
+    /// number of rows produced
+    num_output_rows: usize,
+    /// total time for column unnesting
+    unnest_time: usize,
+}
+
+impl RecordBatchStream for UnnestStream {
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+}
+
+#[async_trait]
+impl Stream for UnnestStream {
+    type Item = Result<RecordBatch>;
+
+    fn poll_next(
+        mut self: std::pin::Pin<&mut Self>,
+        cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Option<Self::Item>> {
+        self.poll_next_impl(cx)
+    }
+}
+
+impl UnnestStream {
+    /// Separate implementation function that unpins the [`UnnestStream`] so
+    /// that partial borrows work correctly
+    fn poll_next_impl(
+        &mut self,
+        cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Option<Result<RecordBatch>>> {
+        self.input
+            .poll_next_unpin(cx)
+            .map(|maybe_batch| match maybe_batch {
+                Some(Ok(batch)) => {
+                    let start = Instant::now();
+                    let result = build_batch(&batch, &self.schema, &self.column);
+                    self.num_input_batches += 1;
+                    self.num_input_rows += batch.num_rows();
+                    if let Ok(ref batch) = result {
+                        self.unnest_time += start.elapsed().as_millis() as usize;
+                        self.num_output_batches += 1;
+                        self.num_output_rows += batch.num_rows();
+                    }
+
+                    Some(result)
+                }
+                other => {
+                    debug!(
+                        "Processed {} probe-side input batches containing {} rows and \
+                        produced {} output batches containing {} rows in {} ms",
+                        self.num_input_batches,
+                        self.num_input_rows,
+                        self.num_output_batches,
+                        self.num_output_rows,
+                        self.unnest_time,
+                    );
+                    other
+                }
+            })
+    }
+}
+
+fn build_batch(
+    batch: &RecordBatch,
+    schema: &SchemaRef,
+    column: &Column,
+) -> Result<RecordBatch> {
+    let list_array = column.evaluate(batch)?.into_array(batch.num_rows());
+    match list_array.data_type() {
+        arrow::datatypes::DataType::List(_) => {
+            let list_array = list_array.as_any().downcast_ref::<ListArray>().unwrap();
+            unnest_batch(batch, schema, column, &list_array)
+        }
+        arrow::datatypes::DataType::LargeList(_) => {
+            let list_array = list_array
+                .as_any()
+                .downcast_ref::<LargeListArray>()
+                .unwrap();
+            unnest_batch(batch, schema, column, &list_array)
+        }
+        arrow::datatypes::DataType::FixedSizeList(_, _) => {
+            let list_array = list_array
+                .as_any()
+                .downcast_ref::<FixedSizeListArray>()
+                .unwrap();
+            unnest_batch(batch, schema, column, list_array)
+        }
+        _ => {
+            return Err(DataFusionError::Execution(format!(
+                "Invalid unnest column {column}"
+            )));
+        }
+    }
+}
+
+fn unnest_batch<T>(
+    batch: &RecordBatch,
+    schema: &SchemaRef,
+    column: &Column,
+    list_array: &T,
+) -> Result<RecordBatch>
+where
+    T: ArrayAccessor<Item = ArrayRef>,
+{
+    let mut batches = Vec::new();
+    let mut num_rows = 0;
+
+    for row in 0..batch.num_rows() {
+        let arrays = batch
+            .columns()
+            .iter()
+            .enumerate()
+            .map(|(col_idx, arr)| {
+                if col_idx == column.index() {
+                    // Unnest the value at the given row.
+                    if list_array.value(row).is_empty() {
+                        // If nested array is empty add an array with 1 null.
+                        Ok(new_null_array(list_array.value(row).data_type(), 1))
+                    } else {
+                        Ok(list_array.value(row))
+                    }
+                } else {
+                    // Number of elements to duplicate, use max(1) to handle null.
+                    let nested_len = list_array.value(row).len().max(1);
+                    // Duplicate rows for each value in the nested array.
+                    if arr.is_null(row) {
+                        Ok(new_null_array(arr.data_type(), nested_len))
+                    } else {
+                        let scalar = ScalarValue::try_from_array(arr, row)?;
+                        Ok(scalar.to_array_of_size(nested_len))
+                    }
+                }
+            })
+            .collect::<Result<Vec<_>>>()?;
+
+        let rb = RecordBatch::try_new(schema.clone(), arrays.to_vec())?;
+        num_rows += rb.num_rows();
+        batches.push(rb);
+    }
+
+    concat_batches(schema, &batches, num_rows).map_err(Into::into)

Review Comment:
   this also results in another copy -- aka it is an optimization for the future



##########
datafusion/core/tests/dataframe.rs:
##########
@@ -500,6 +503,110 @@ async fn right_anti_filter_push_down() -> Result<()> {
     Ok(())
 }
 
+#[tokio::test]
+async fn unnest_columns() -> Result<()> {
+    const NUM_ROWS: usize = 4;
+    let df = table_with_nested_types(NUM_ROWS).await?;
+    let results = df.collect().await?;
+    let expected = vec![
+        r#"+----------+------------------------------------------------------------+--------------------+"#,
+        r#"| shape_id | points                                                     | tags               |"#,
+        r#"+----------+------------------------------------------------------------+--------------------+"#,
+        r#"| 1        | [{"x": -3, "y": -4}, {"x": -3, "y": 6}, {"x": 2, "y": -2}] | [tag1]             |"#,
+        r#"| 2        |                                                            | [tag1, tag2]       |"#,
+        r#"| 3        | [{"x": -9, "y": 2}, {"x": -10, "y": -4}]                   |                    |"#,
+        r#"| 4        | [{"x": -3, "y": 5}, {"x": 2, "y": -1}]                     | [tag1, tag2, tag3] |"#,
+        r#"+----------+------------------------------------------------------------+--------------------+"#,
+    ];
+    assert_batches_sorted_eq!(expected, &results);
+
+    // Unnest tags
+    let df = table_with_nested_types(NUM_ROWS).await?;
+    let results = df.unnest_column("tags")?.collect().await?;
+    let expected = vec![
+        r#"+----------+------------------------------------------------------------+------+"#,
+        r#"| shape_id | points                                                     | tags |"#,
+        r#"+----------+------------------------------------------------------------+------+"#,
+        r#"| 1        | [{"x": -3, "y": -4}, {"x": -3, "y": 6}, {"x": 2, "y": -2}] | tag1 |"#,
+        r#"| 2        |                                                            | tag1 |"#,
+        r#"| 2        |                                                            | tag2 |"#,
+        r#"| 3        | [{"x": -9, "y": 2}, {"x": -10, "y": -4}]                   |      |"#,
+        r#"| 4        | [{"x": -3, "y": 5}, {"x": 2, "y": -1}]                     | tag1 |"#,
+        r#"| 4        | [{"x": -3, "y": 5}, {"x": 2, "y": -1}]                     | tag2 |"#,
+        r#"| 4        | [{"x": -3, "y": 5}, {"x": 2, "y": -1}]                     | tag3 |"#,
+        r#"+----------+------------------------------------------------------------+------+"#,
+    ];
+    assert_batches_sorted_eq!(expected, &results);
+
+    // Test aggregate results for tags.
+    let df = table_with_nested_types(NUM_ROWS).await?;
+    let count = df.unnest_column("tags")?.count().await?;
+    assert_eq!(count, results.iter().map(|r| r.num_rows()).sum::<usize>());
+
+    // Unnest points
+    let df = table_with_nested_types(NUM_ROWS).await?;
+    let results = df.unnest_column("points")?.collect().await?;
+    let expected = vec![
+        r#"+----------+---------------------+--------------------+"#,
+        r#"| shape_id | points              | tags               |"#,
+        r#"+----------+---------------------+--------------------+"#,
+        r#"| 1        | {"x": -3, "y": -4}  | [tag1]             |"#,
+        r#"| 1        | {"x": -3, "y": 6}   | [tag1]             |"#,
+        r#"| 1        | {"x": 2, "y": -2}   | [tag1]             |"#,
+        r#"| 2        |                     | [tag1, tag2]       |"#,
+        r#"| 3        | {"x": -9, "y": 2}   |                    |"#,
+        r#"| 3        | {"x": -10, "y": -4} |                    |"#,
+        r#"| 4        | {"x": -3, "y": 5}   | [tag1, tag2, tag3] |"#,
+        r#"| 4        | {"x": 2, "y": -1}   | [tag1, tag2, tag3] |"#,
+        r#"+----------+---------------------+--------------------+"#,
+    ];
+    assert_batches_sorted_eq!(expected, &results);
+
+    // Test aggregate results for points.
+    let df = table_with_nested_types(NUM_ROWS).await?;
+    let count = df.unnest_column("points")?.count().await?;
+    assert_eq!(count, results.iter().map(|r| r.num_rows()).sum::<usize>());
+
+    // Unnest both points and tags.
+    let df = table_with_nested_types(NUM_ROWS).await?;
+    let results = df
+        .unnest_column("points")?
+        .unnest_column("tags")?
+        .collect()
+        .await?;
+    let expected = vec![
+        r#"+----------+---------------------+------+"#,
+        r#"| shape_id | points              | tags |"#,
+        r#"+----------+---------------------+------+"#,
+        r#"| 1        | {"x": -3, "y": -4}  | tag1 |"#,
+        r#"| 1        | {"x": -3, "y": 6}   | tag1 |"#,
+        r#"| 1        | {"x": 2, "y": -2}   | tag1 |"#,
+        r#"| 2        |                     | tag1 |"#,
+        r#"| 2        |                     | tag2 |"#,
+        r#"| 3        | {"x": -9, "y": 2}   |      |"#,
+        r#"| 3        | {"x": -10, "y": -4} |      |"#,
+        r#"| 4        | {"x": -3, "y": 5}   | tag1 |"#,
+        r#"| 4        | {"x": -3, "y": 5}   | tag2 |"#,
+        r#"| 4        | {"x": -3, "y": 5}   | tag3 |"#,
+        r#"| 4        | {"x": 2, "y": -1}   | tag1 |"#,
+        r#"| 4        | {"x": 2, "y": -1}   | tag2 |"#,
+        r#"| 4        | {"x": 2, "y": -1}   | tag3 |"#,
+        r#"+----------+---------------------+------+"#,
+    ];
+    assert_batches_sorted_eq!(expected, &results);
+
+    // Test aggregate results for points and tags.

Review Comment:
   👍  nice test coverage



##########
datafusion/expr/src/utils.rs:
##########
@@ -739,6 +739,35 @@ pub fn from_plan(
             Ok(plan.clone())
         }
         LogicalPlan::DescribeTable(_) => Ok(plan.clone()),
+        LogicalPlan::Unnest(Unnest { column, schema, .. }) => {
+            // Update schema with unnested column type.
+            let input = Arc::new(inputs[0].clone());
+            let nested_field = input.schema().field_from_column(column)?;
+            let unnested_field = schema.field_from_column(column)?;
+            let fields = input
+                .schema()
+                .fields()
+                .iter()
+                .map(|f| {
+                    if f == nested_field {
+                        unnested_field.clone()
+                    } else {
+                        f.clone()
+                    }
+                })
+                .collect::<Vec<_>>();
+
+            let schema = Arc::new(DFSchema::new_with_metadata(
+                fields,
+                input.schema().metadata().clone(),
+            )?);
+
+            Ok(LogicalPlan::Unnest(Unnest {
+                input,
+                column: column.clone(),
+                schema,
+            }))
+        }

Review Comment:
   I think the intent of `from_plan` is to be mechanical construction rather than potentially changing the schema, so I was surprised this code is necessary.
   
   However, when I tried changing this code to be something more like
   
   ```rust
               Ok(LogicalPlan::Unnest(Unnest {
                   input: inputs[0].clone().into(),
                   column: column.clone(),
                   schema: schema.clone(),
               }))
   ```
   
   The test fails:
   
   ```
   
   ---- unnest_columns stdout ----
   Error: Internal("PhysicalExpr Column references column 'tags' at index 2 (zero-based) but input schema only has 1 columns: [\"tags\"]")
   
   
   failures:
       unnest_columns
   ```
   
   



##########
datafusion/core/src/physical_plan/unnest.rs:
##########
@@ -0,0 +1,305 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Defines the unnest column plan for unnesting values in a column that contains a list
+//! type, conceptually is like joining each row with all the values in the list column.
+use arrow::array::{
+    new_null_array, Array, ArrayAccessor, ArrayRef, FixedSizeListArray, LargeListArray,
+    ListArray,
+};
+use arrow::datatypes::{Schema, SchemaRef};
+use arrow::record_batch::RecordBatch;
+use async_trait::async_trait;
+use futures::Stream;
+use futures::StreamExt;
+use log::debug;
+use std::time::Instant;
+use std::{any::Any, sync::Arc};
+
+use crate::execution::context::TaskContext;
+use crate::physical_plan::{
+    coalesce_batches::concat_batches, expressions::Column, DisplayFormatType,
+    Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, PhysicalExpr,
+    PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+use crate::{
+    error::{DataFusionError, Result},
+    scalar::ScalarValue,
+};
+
+/// Unnest the given column by joining the row with each value in the nested type.
+#[derive(Debug)]
+pub struct UnnestExec {
+    /// Input execution plan
+    input: Arc<dyn ExecutionPlan>,
+    /// The schema once the unnest is applied
+    schema: SchemaRef,
+    /// The unnest column
+    column: Column,
+}
+
+impl UnnestExec {
+    /// Create a new [UnnestExec].
+    pub fn new(input: Arc<dyn ExecutionPlan>, column: Column, schema: SchemaRef) -> Self {
+        UnnestExec {
+            input,
+            schema,
+            column,
+        }
+    }
+}
+
+impl ExecutionPlan for UnnestExec {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+
+    fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
+        vec![self.input.clone()]
+    }
+
+    /// Specifies whether this plan generates an infinite stream of records.
+    /// If the plan does not support pipelining, but it its input(s) are
+    /// infinite, returns an error to indicate this.    
+    fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
+        Ok(children[0])
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> Result<Arc<dyn ExecutionPlan>> {
+        Ok(Arc::new(UnnestExec::new(
+            children[0].clone(),
+            self.column.clone(),
+            self.schema.clone(),
+        )))
+    }
+
+    fn required_input_distribution(&self) -> Vec<Distribution> {
+        vec![Distribution::UnspecifiedDistribution]
+    }
+
+    fn output_partitioning(&self) -> Partitioning {
+        self.input.output_partitioning()
+    }
+
+    fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
+        None
+    }
+
+    fn equivalence_properties(&self) -> EquivalenceProperties {
+        self.input.equivalence_properties()
+    }
+
+    fn execute(
+        &self,
+        partition: usize,
+        context: Arc<TaskContext>,
+    ) -> Result<SendableRecordBatchStream> {
+        let input = self.input.execute(partition, context)?;
+
+        Ok(Box::pin(UnnestStream {
+            input,
+            schema: self.schema.clone(),
+            column: self.column.clone(),
+            num_input_batches: 0,
+            num_input_rows: 0,
+            num_output_batches: 0,
+            num_output_rows: 0,
+            unnest_time: 0,
+        }))
+    }
+
+    fn fmt_as(
+        &self,
+        t: DisplayFormatType,
+        f: &mut std::fmt::Formatter,
+    ) -> std::fmt::Result {
+        match t {
+            DisplayFormatType::Default => {
+                write!(f, "UnnestExec")
+            }
+        }
+    }
+
+    fn statistics(&self) -> Statistics {
+        self.input.statistics()
+    }
+}
+
+/// A stream that issues [RecordBatch]es with unnested column data.
+struct UnnestStream {
+    /// Input stream
+    input: SendableRecordBatchStream,
+    /// Unnested schema
+    schema: Arc<Schema>,
+    /// The unnest column
+    column: Column,
+    /// number of input batches
+    num_input_batches: usize,
+    /// number of input rows
+    num_input_rows: usize,
+    /// number of batches produced
+    num_output_batches: usize,
+    /// number of rows produced
+    num_output_rows: usize,
+    /// total time for column unnesting
+    unnest_time: usize,
+}
+
+impl RecordBatchStream for UnnestStream {
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+}
+
+#[async_trait]
+impl Stream for UnnestStream {
+    type Item = Result<RecordBatch>;
+
+    fn poll_next(
+        mut self: std::pin::Pin<&mut Self>,
+        cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Option<Self::Item>> {
+        self.poll_next_impl(cx)
+    }
+}
+
+impl UnnestStream {
+    /// Separate implementation function that unpins the [`UnnestStream`] so
+    /// that partial borrows work correctly
+    fn poll_next_impl(
+        &mut self,
+        cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Option<Result<RecordBatch>>> {
+        self.input
+            .poll_next_unpin(cx)
+            .map(|maybe_batch| match maybe_batch {
+                Some(Ok(batch)) => {
+                    let start = Instant::now();
+                    let result = build_batch(&batch, &self.schema, &self.column);
+                    self.num_input_batches += 1;
+                    self.num_input_rows += batch.num_rows();
+                    if let Ok(ref batch) = result {
+                        self.unnest_time += start.elapsed().as_millis() as usize;
+                        self.num_output_batches += 1;
+                        self.num_output_rows += batch.num_rows();
+                    }
+
+                    Some(result)
+                }
+                other => {
+                    debug!(
+                        "Processed {} probe-side input batches containing {} rows and \
+                        produced {} output batches containing {} rows in {} ms",
+                        self.num_input_batches,
+                        self.num_input_rows,
+                        self.num_output_batches,
+                        self.num_output_rows,
+                        self.unnest_time,
+                    );
+                    other
+                }
+            })
+    }
+}
+
+fn build_batch(
+    batch: &RecordBatch,
+    schema: &SchemaRef,
+    column: &Column,
+) -> Result<RecordBatch> {
+    let list_array = column.evaluate(batch)?.into_array(batch.num_rows());
+    match list_array.data_type() {
+        arrow::datatypes::DataType::List(_) => {
+            let list_array = list_array.as_any().downcast_ref::<ListArray>().unwrap();
+            unnest_batch(batch, schema, column, &list_array)
+        }
+        arrow::datatypes::DataType::LargeList(_) => {
+            let list_array = list_array
+                .as_any()
+                .downcast_ref::<LargeListArray>()
+                .unwrap();
+            unnest_batch(batch, schema, column, &list_array)
+        }
+        arrow::datatypes::DataType::FixedSizeList(_, _) => {
+            let list_array = list_array
+                .as_any()
+                .downcast_ref::<FixedSizeListArray>()
+                .unwrap();
+            unnest_batch(batch, schema, column, list_array)
+        }
+        _ => {
+            return Err(DataFusionError::Execution(format!(
+                "Invalid unnest column {column}"
+            )));
+        }
+    }
+}
+
+fn unnest_batch<T>(
+    batch: &RecordBatch,
+    schema: &SchemaRef,
+    column: &Column,
+    list_array: &T,
+) -> Result<RecordBatch>
+where
+    T: ArrayAccessor<Item = ArrayRef>,
+{
+    let mut batches = Vec::new();
+    let mut num_rows = 0;
+
+    for row in 0..batch.num_rows() {
+        let arrays = batch
+            .columns()
+            .iter()
+            .enumerate()
+            .map(|(col_idx, arr)| {
+                if col_idx == column.index() {
+                    // Unnest the value at the given row.
+                    if list_array.value(row).is_empty() {
+                        // If nested array is empty add an array with 1 null.
+                        Ok(new_null_array(list_array.value(row).data_type(), 1))
+                    } else {
+                        Ok(list_array.value(row))
+                    }
+                } else {
+                    // Number of elements to duplicate, use max(1) to handle null.
+                    let nested_len = list_array.value(row).len().max(1);
+                    // Duplicate rows for each value in the nested array.
+                    if arr.is_null(row) {
+                        Ok(new_null_array(arr.data_type(), nested_len))
+                    } else {
+                        let scalar = ScalarValue::try_from_array(arr, row)?;
+                        Ok(scalar.to_array_of_size(nested_len))

Review Comment:
   Copying via ScalarValue I think is fine for the initial implementation
   
   To make this faster, I think we could probably use the take kernel https://docs.rs/arrow/32.0.0/arrow/compute/kernels/take/fn.take.html as it is quite a bit faster to copy values from one array to another



-- 
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-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org