Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

finish implementation of window accumulation and func row_number() and add sql.rs test #413

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 7 additions & 0 deletions datafusion/src/execution/context.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1268,6 +1268,13 @@ mod tests {
Ok(())
}

#[tokio::test]
async fn window() -> Result<()> {
let results = execute("SELECT c1, MAX(c2) OVER () FROM test", 4).await?;
assert_eq!(results.len(), 1);
Ok(())
}

#[tokio::test]
async fn aggregate() -> Result<()> {
let results = execute("SELECT SUM(c1), SUM(c2) FROM test", 4).await?;
Expand Down
4 changes: 4 additions & 0 deletions datafusion/src/physical_plan/expressions/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,9 @@ mod literal;
mod min_max;
mod negative;
mod not;
mod nth_value;
mod nullif;
mod row_number;
mod sum;
mod try_cast;

Expand All @@ -57,7 +59,9 @@ pub use literal::{lit, Literal};
pub use min_max::{Max, Min};
pub use negative::{negative, NegativeExpr};
pub use not::{not, NotExpr};
pub use nth_value::{FirstValue, LastValue, NthValue};
pub use nullif::{nullif_func, SUPPORTED_NULLIF_TYPES};
pub use row_number::RowNumber;
pub use sum::{sum_return_type, Sum};
pub use try_cast::{try_cast, TryCastExpr};
/// returns the name of the state
Expand Down
221 changes: 221 additions & 0 deletions datafusion/src/physical_plan/expressions/nth_value.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,221 @@
// 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 physical expressions that can evaluated at runtime during query execution

use crate::error::{DataFusionError, Result};
use crate::physical_plan::{BuiltInWindowFunctionExpr, PhysicalExpr, WindowAccumulator};
use crate::scalar::ScalarValue;
use arrow::datatypes::{DataType, Field};
use std::any::Any;
use std::convert::TryFrom;
use std::sync::Arc;

/// first_value expression
#[derive(Debug)]
pub struct FirstValue {
name: String,
data_type: DataType,
expr: Arc<dyn PhysicalExpr>,
}

impl FirstValue {
/// Create a new FIRST_VALUE window aggregate function
pub fn new(expr: Arc<dyn PhysicalExpr>, name: String, data_type: DataType) -> Self {
Self {
name,
data_type,
expr,
}
}
}

impl BuiltInWindowFunctionExpr for FirstValue {
/// Return a reference to Any that can be used for downcasting
fn as_any(&self) -> &dyn Any {
self
}

fn field(&self) -> Result<Field> {
let nullable = true;
Ok(Field::new(&self.name, self.data_type.clone(), nullable))
}

fn expressions(&self) -> Vec<Arc<dyn PhysicalExpr>> {
vec![self.expr.clone()]
}

fn name(&self) -> &str {
&self.name
}

fn create_accumulator(&self) -> Result<Box<dyn WindowAccumulator>> {
Ok(Box::new(NthValueAccumulator::try_new(
1,
self.data_type.clone(),
)?))
}
}

// sql values start with 1, so we can use 0 to indicate the special last value behavior
const SPECIAL_SIZE_VALUE_FOR_LAST: u32 = 0;

/// last_value expression
#[derive(Debug)]
pub struct LastValue {
name: String,
data_type: DataType,
expr: Arc<dyn PhysicalExpr>,
}

impl LastValue {
/// Create a new FIRST_VALUE window aggregate function
pub fn new(expr: Arc<dyn PhysicalExpr>, name: String, data_type: DataType) -> Self {
Self {
name,
data_type,
expr,
}
}
}

impl BuiltInWindowFunctionExpr for LastValue {
/// Return a reference to Any that can be used for downcasting
fn as_any(&self) -> &dyn Any {
self
}

fn field(&self) -> Result<Field> {
let nullable = true;
Ok(Field::new(&self.name, self.data_type.clone(), nullable))
}

fn expressions(&self) -> Vec<Arc<dyn PhysicalExpr>> {
vec![self.expr.clone()]
}

fn name(&self) -> &str {
&self.name
}

fn create_accumulator(&self) -> Result<Box<dyn WindowAccumulator>> {
Ok(Box::new(NthValueAccumulator::try_new(
SPECIAL_SIZE_VALUE_FOR_LAST,
self.data_type.clone(),
)?))
}
}

/// nth_value expression
#[derive(Debug)]
pub struct NthValue {
name: String,
n: u32,
data_type: DataType,
expr: Arc<dyn PhysicalExpr>,
}

impl NthValue {
/// Create a new NTH_VALUE window aggregate function
pub fn try_new(
expr: Arc<dyn PhysicalExpr>,
name: String,
n: u32,
data_type: DataType,
) -> Result<Self> {
if n == SPECIAL_SIZE_VALUE_FOR_LAST {
Err(DataFusionError::Execution(
"nth_value expect n to be > 0".to_owned(),
))
} else {
Ok(Self {
name,
n,
data_type,
expr,
})
}
}
}

impl BuiltInWindowFunctionExpr for NthValue {
/// Return a reference to Any that can be used for downcasting
fn as_any(&self) -> &dyn Any {
self
}

fn field(&self) -> Result<Field> {
let nullable = true;
Ok(Field::new(&self.name, self.data_type.clone(), nullable))
}

fn expressions(&self) -> Vec<Arc<dyn PhysicalExpr>> {
vec![self.expr.clone()]
}

fn name(&self) -> &str {
&self.name
}

fn create_accumulator(&self) -> Result<Box<dyn WindowAccumulator>> {
Ok(Box::new(NthValueAccumulator::try_new(
self.n,
self.data_type.clone(),
)?))
}
}

#[derive(Debug)]
struct NthValueAccumulator {
// n the target nth_value, however we'll reuse it for last_value acc, so when n == 0 it specifically
// means last; also note that it is totally valid for n to be larger than the number of rows input
// in which case all the values shall be null
n: u32,
offset: u32,
value: ScalarValue,
}

impl NthValueAccumulator {
/// new count accumulator
pub fn try_new(n: u32, data_type: DataType) -> Result<Self> {
Ok(Self {
n,
offset: 0,
// null value of that data_type by default
value: ScalarValue::try_from(&data_type)?,
})
}
}

impl WindowAccumulator for NthValueAccumulator {
fn scan(&mut self, values: &[ScalarValue]) -> Result<Option<ScalarValue>> {
if self.n == SPECIAL_SIZE_VALUE_FOR_LAST {
// for last_value function
self.value = values[0].clone();
} else if self.offset < self.n {
self.offset += 1;
if self.offset == self.n {
self.value = values[0].clone();
}
}
Ok(None)
}

fn evaluate(&self) -> Result<Option<ScalarValue>> {
Ok(Some(self.value.clone()))
}
}
102 changes: 102 additions & 0 deletions datafusion/src/physical_plan/expressions/row_number.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,102 @@
// 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 physical expressions that can evaluated at runtime during query execution

use crate::error::Result;
use crate::physical_plan::{BuiltInWindowFunctionExpr, PhysicalExpr, WindowAccumulator};
use crate::scalar::ScalarValue;
use arrow::array::ArrayRef;
use arrow::datatypes::{DataType, Field};
use std::any::Any;
use std::sync::Arc;

/// row_number expression
#[derive(Debug)]
pub struct RowNumber {
name: String,
}

impl RowNumber {
/// Create a new MAX aggregate function
pub fn new(name: String) -> Self {
Self { name }
}
}

impl BuiltInWindowFunctionExpr for RowNumber {
/// Return a reference to Any that can be used for downcasting
fn as_any(&self) -> &dyn Any {
self
}

fn field(&self) -> Result<Field> {
let nullable = false;
let data_type = DataType::UInt64;
Ok(Field::new(&self.name, data_type, nullable))
}

fn expressions(&self) -> Vec<Arc<dyn PhysicalExpr>> {
vec![]
}

fn name(&self) -> &str {
&self.name
}

fn create_accumulator(&self) -> Result<Box<dyn WindowAccumulator>> {
Ok(Box::new(RowNumberAccumulator::new()))
}
}

#[derive(Debug)]
struct RowNumberAccumulator {
row_number: u64,
}

impl RowNumberAccumulator {
/// new count accumulator
pub fn new() -> Self {
// row number is 1 based
Self { row_number: 1 }
}
}

impl WindowAccumulator for RowNumberAccumulator {
fn scan(&mut self, _values: &[ScalarValue]) -> Result<Option<ScalarValue>> {
let result = Some(ScalarValue::UInt64(Some(self.row_number)));
self.row_number += 1;
Ok(result)
}

fn scan_batch(
&mut self,
num_rows: usize,
_values: &[ArrayRef],
) -> Result<Option<Vec<ScalarValue>>> {
let new_row_number = self.row_number + (num_rows as u64);
let result = (self.row_number..new_row_number)
.map(|i| ScalarValue::UInt64(Some(i)))
.collect();
self.row_number = new_row_number;
Ok(Some(result))
}

fn evaluate(&self) -> Result<Option<ScalarValue>> {
Ok(None)
}
}
7 changes: 4 additions & 3 deletions datafusion/src/physical_plan/hash_aggregate.rs
Original file line number Diff line number Diff line change
Expand Up @@ -712,7 +712,7 @@ impl GroupedHashAggregateStream {
tx.send(result)
});

GroupedHashAggregateStream {
Self {
schema,
output: rx,
finished: false,
Expand Down Expand Up @@ -825,7 +825,8 @@ fn aggregate_expressions(
}

pin_project! {
struct HashAggregateStream {
/// stream struct for hash aggregation
pub struct HashAggregateStream {
schema: SchemaRef,
#[pin]
output: futures::channel::oneshot::Receiver<ArrowResult<RecordBatch>>,
Expand Down Expand Up @@ -878,7 +879,7 @@ impl HashAggregateStream {
tx.send(result)
});

HashAggregateStream {
Self {
schema,
output: rx,
finished: false,
Expand Down
Loading