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/06/09 18:08:10 UTC

[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #6617: RFC: User Defined Window Functions

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


##########
datafusion-examples/examples/simple_udwf.rs:
##########
@@ -0,0 +1,210 @@
+// 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.
+
+use std::sync::Arc;
+
+use arrow::{
+    array::{AsArray, Float64Array},
+    datatypes::Float64Type,
+};
+use arrow_schema::DataType;
+use datafusion::datasource::file_format::options::CsvReadOptions;
+
+use datafusion::error::Result;
+use datafusion::prelude::*;
+use datafusion_common::DataFusionError;
+use datafusion_expr::{
+    partition_evaluator::PartitionEvaluator, Signature, Volatility, WindowUDF,
+};
+
+// create local execution context with `cars.csv` registered as a table named `cars`
+async fn create_context() -> Result<SessionContext> {
+    // declare a new context. In spark API, this corresponds to a new spark SQLsession
+    let ctx = SessionContext::new();
+
+    // declare a table in memory. In spark API, this corresponds to createDataFrame(...).
+    println!("pwd: {}", std::env::current_dir().unwrap().display());
+    let csv_path = format!("datafusion/core/tests/data/cars.csv");
+    let read_options = CsvReadOptions::default().has_header(true);
+
+    ctx.register_csv("cars", &csv_path, read_options).await?;
+    Ok(ctx)
+}
+
+/// In this example we will declare a user defined window function that computes a moving average and then run it using SQL
+#[tokio::main]
+async fn main() -> Result<()> {
+    let ctx = create_context().await?;
+
+    // register the window function with DataFusion so wecan call it
+    ctx.register_udwf(my_average());
+
+    // Use SQL to run the new window function
+    let df = ctx.sql("SELECT * from cars").await?;
+    // print the results
+    df.show().await?;
+
+    // Use SQL to run the new window function
+    // `PARTITION BY car`:each distinct value of car (red, and green) should be treated separately
+    // `ORDER BY time`: within each group (greed or green) the values will be orderd by time
+    let df = ctx
+        .sql(
+            "SELECT car, \
+                      speed, \
+                      lag(speed, 1) OVER (PARTITION BY car ORDER BY time),\
+                      my_average(speed) OVER (PARTITION BY car ORDER BY time),\

Review Comment:
   This shows calling the user defined window function via SQL



##########
datafusion-examples/examples/simple_udwf.rs:
##########
@@ -0,0 +1,210 @@
+// 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.
+
+use std::sync::Arc;
+
+use arrow::{
+    array::{AsArray, Float64Array},
+    datatypes::Float64Type,
+};
+use arrow_schema::DataType;
+use datafusion::datasource::file_format::options::CsvReadOptions;
+
+use datafusion::error::Result;
+use datafusion::prelude::*;
+use datafusion_common::DataFusionError;
+use datafusion_expr::{
+    partition_evaluator::PartitionEvaluator, Signature, Volatility, WindowUDF,
+};
+
+// create local execution context with `cars.csv` registered as a table named `cars`
+async fn create_context() -> Result<SessionContext> {
+    // declare a new context. In spark API, this corresponds to a new spark SQLsession
+    let ctx = SessionContext::new();
+
+    // declare a table in memory. In spark API, this corresponds to createDataFrame(...).
+    println!("pwd: {}", std::env::current_dir().unwrap().display());
+    let csv_path = format!("datafusion/core/tests/data/cars.csv");
+    let read_options = CsvReadOptions::default().has_header(true);
+
+    ctx.register_csv("cars", &csv_path, read_options).await?;
+    Ok(ctx)
+}
+
+/// In this example we will declare a user defined window function that computes a moving average and then run it using SQL
+#[tokio::main]
+async fn main() -> Result<()> {
+    let ctx = create_context().await?;
+
+    // register the window function with DataFusion so wecan call it
+    ctx.register_udwf(my_average());
+
+    // Use SQL to run the new window function
+    let df = ctx.sql("SELECT * from cars").await?;
+    // print the results
+    df.show().await?;
+
+    // Use SQL to run the new window function
+    // `PARTITION BY car`:each distinct value of car (red, and green) should be treated separately
+    // `ORDER BY time`: within each group (greed or green) the values will be orderd by time
+    let df = ctx
+        .sql(
+            "SELECT car, \
+                      speed, \
+                      lag(speed, 1) OVER (PARTITION BY car ORDER BY time),\
+                      my_average(speed) OVER (PARTITION BY car ORDER BY time),\
+                      time \
+                      from cars",
+        )
+        .await?;
+    // print the results
+    df.show().await?;
+
+    // // ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING: Run the window functon so that each invocation only sees 5 rows: the 2 before and 2 after) using
+    // let df = ctx.sql("SELECT car, \
+    //                   speed, \
+    //                   lag(speed, 1) OVER (PARTITION BY car ORDER BY time ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING),\
+    //                   time \
+    //                   from cars").await?;
+    // // print the results
+    // df.show().await?;
+
+    // todo show how to run dataframe API as well
+
+    Ok(())
+}
+
+// TODO make a helper funciton like `crate_udf` that helps to make these signatures
+
+fn my_average() -> WindowUDF {
+    WindowUDF {
+        name: String::from("my_average"),
+        // it will take 2 arguments -- the column and the window size
+        signature: Signature::exact(vec![DataType::Int32], Volatility::Immutable),
+        return_type: Arc::new(return_type),
+        partition_evaluator: Arc::new(make_partition_evaluator),
+    }
+}
+
+/// Compute the return type of the function given the argument types
+fn return_type(arg_types: &[DataType]) -> Result<Arc<DataType>> {
+    if arg_types.len() != 1 {
+        return Err(DataFusionError::Plan(format!(
+            "my_udwf expects 1 argument, got {}: {:?}",
+            arg_types.len(),
+            arg_types
+        )));
+    }
+    Ok(Arc::new(arg_types[0].clone()))
+}
+
+/// Create a partition evaluator for this argument
+fn make_partition_evaluator() -> Result<Box<dyn PartitionEvaluator>> {
+    Ok(Box::new(MyPartitionEvaluator::new()))
+}
+
+/// This implements the lowest level evaluation for a window function
+///
+/// It handles calculating the value of the window function for each
+/// distinct values of `PARTITION BY` (each car type in our example)
+#[derive(Clone, Debug)]
+struct MyPartitionEvaluator {}
+
+impl MyPartitionEvaluator {
+    fn new() -> Self {
+        Self {}
+    }
+}
+
+/// These different evaluation methods are called depending on the various settings of WindowUDF
+impl PartitionEvaluator for MyPartitionEvaluator {

Review Comment:
   Here is the proposal of how a user would specify specify the window calculation -- by  `impl PartitionEvaluator`



##########
datafusion/expr/src/udwf.rs:
##########
@@ -0,0 +1,83 @@
+// 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.
+
+//! Support for user-defined window (UDWF) window functions
+
+use std::fmt::{self, Debug, Display, Formatter};
+
+use crate::{function::PartitionEvaluatorFunctionFactory, ReturnTypeFunction, Signature};
+
+/// Logical representation of a user-defined window function (UDWF)
+/// A UDAF is different from a UDF in that it is stateful across batches.
+#[derive(Clone)]

Review Comment:
   here is the proposed interface for defining a WindowUDF -- it is very similar to ScalarUDF and AggregateUDF, on purpose



##########
datafusion/expr/src/partition_evaluator.rs:
##########
@@ -17,14 +17,24 @@
 
 //! Partition evaluation module
 
-use crate::window::window_expr::BuiltinWindowState;
-use crate::window::WindowAggState;
+use crate::window_frame_state::WindowAggState;

Review Comment:
   This file is moved into `datafusion_expr` and I had to make a few small changes related to state management -- but I think I may be able to avoid that.



##########
datafusion/core/src/physical_plan/windows/mod.rs:
##########
@@ -184,6 +190,67 @@ fn create_built_in_window_expr(
     })
 }
 
+/// Creates a `BuiltInWindowFunctionExpr` suitable for a user defined window function
+fn create_udwf_window_expr(
+    fun: &Arc<WindowUDF>,
+    args: &[Arc<dyn PhysicalExpr>],
+    input_schema: &Schema,
+    name: String,
+) -> Result<Arc<dyn BuiltInWindowFunctionExpr>> {
+    // need to get the types into an owned vec for some reason
+    let input_types: Vec<_> = args
+        .iter()
+        .map(|arg| arg.data_type(input_schema).map(|dt| dt.clone()))
+        .collect::<Result<_>>()?;
+
+    // figure out the output type
+    let data_type = (fun.return_type)(&input_types)?;
+    Ok(Arc::new(WindowUDFExpr {
+        fun: Arc::clone(fun),
+        args: args.to_vec(),
+        name,
+        data_type,
+    }))
+}
+
+// Implement BuiltInWindowFunctionExpr for WindowUDF
+#[derive(Clone, Debug)]
+struct WindowUDFExpr {
+    fun: Arc<WindowUDF>,
+    args: Vec<Arc<dyn PhysicalExpr>>,
+    /// Display name
+    name: String,
+    /// result type
+    data_type: Arc<DataType>,
+}
+
+impl BuiltInWindowFunctionExpr for WindowUDFExpr {

Review Comment:
   This thing adapts a `WindowUDF` to what the actual WindowExec's need



##########
datafusion/expr/src/window_frame_state.rs:
##########
@@ -18,17 +18,112 @@
 //! This module provides utilities for window frame index calculations
 //! depending on the window frame mode: RANGE, ROWS, GROUPS.
 
+use crate::{WindowFrame, WindowFrameBound, WindowFrameUnits};
 use arrow::array::ArrayRef;
+use arrow::compute::concat;
 use arrow::compute::kernels::sort::SortOptions;
+use arrow::datatypes::DataType;
+use arrow::record_batch::RecordBatch;
 use datafusion_common::utils::{compare_rows, get_row_at_idx, search_in_slice};
 use datafusion_common::{DataFusionError, Result, ScalarValue};
-use datafusion_expr::{WindowFrame, WindowFrameBound, WindowFrameUnits};
 use std::cmp::min;
 use std::collections::VecDeque;
 use std::fmt::Debug;
 use std::ops::Range;
 use std::sync::Arc;
 
+/// State for each unique partition determined according to PARTITION BY column(s)

Review Comment:
   this stuff is related to state management and was just moved (not modified)



##########
datafusion-examples/examples/simple_udwf.rs:
##########
@@ -0,0 +1,210 @@
+// 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.
+
+use std::sync::Arc;
+
+use arrow::{
+    array::{AsArray, Float64Array},
+    datatypes::Float64Type,
+};
+use arrow_schema::DataType;
+use datafusion::datasource::file_format::options::CsvReadOptions;
+
+use datafusion::error::Result;
+use datafusion::prelude::*;
+use datafusion_common::DataFusionError;
+use datafusion_expr::{
+    partition_evaluator::PartitionEvaluator, Signature, Volatility, WindowUDF,
+};
+
+// create local execution context with `cars.csv` registered as a table named `cars`
+async fn create_context() -> Result<SessionContext> {
+    // declare a new context. In spark API, this corresponds to a new spark SQLsession
+    let ctx = SessionContext::new();
+
+    // declare a table in memory. In spark API, this corresponds to createDataFrame(...).
+    println!("pwd: {}", std::env::current_dir().unwrap().display());
+    let csv_path = format!("datafusion/core/tests/data/cars.csv");
+    let read_options = CsvReadOptions::default().has_header(true);
+
+    ctx.register_csv("cars", &csv_path, read_options).await?;
+    Ok(ctx)
+}
+
+/// In this example we will declare a user defined window function that computes a moving average and then run it using SQL
+#[tokio::main]
+async fn main() -> Result<()> {
+    let ctx = create_context().await?;
+
+    // register the window function with DataFusion so wecan call it
+    ctx.register_udwf(my_average());
+
+    // Use SQL to run the new window function
+    let df = ctx.sql("SELECT * from cars").await?;
+    // print the results
+    df.show().await?;
+
+    // Use SQL to run the new window function
+    // `PARTITION BY car`:each distinct value of car (red, and green) should be treated separately
+    // `ORDER BY time`: within each group (greed or green) the values will be orderd by time
+    let df = ctx
+        .sql(
+            "SELECT car, \
+                      speed, \
+                      lag(speed, 1) OVER (PARTITION BY car ORDER BY time),\
+                      my_average(speed) OVER (PARTITION BY car ORDER BY time),\
+                      time \
+                      from cars",
+        )
+        .await?;
+    // print the results
+    df.show().await?;
+
+    // // ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING: Run the window functon so that each invocation only sees 5 rows: the 2 before and 2 after) using
+    // let df = ctx.sql("SELECT car, \
+    //                   speed, \
+    //                   lag(speed, 1) OVER (PARTITION BY car ORDER BY time ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING),\
+    //                   time \
+    //                   from cars").await?;
+    // // print the results
+    // df.show().await?;
+
+    // todo show how to run dataframe API as well
+
+    Ok(())
+}
+
+// TODO make a helper funciton like `crate_udf` that helps to make these signatures
+
+fn my_average() -> WindowUDF {
+    WindowUDF {

Review Comment:
   Here is the structure that provides metadata about the window function



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