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

support for array_repeat array function #1205

Open
wants to merge 17 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 13 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
44 changes: 44 additions & 0 deletions native/core/src/execution/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,7 @@ use datafusion_expr::{
WindowFunctionDefinition,
};
use datafusion_functions_nested::array_has::ArrayHas;
use datafusion_functions_nested::repeat::array_repeat_udf;
use datafusion_physical_expr::expressions::{Literal, StatsType};
use datafusion_physical_expr::window::WindowExpr;
use datafusion_physical_expr::LexOrdering;
Expand Down Expand Up @@ -775,6 +776,49 @@ impl PhysicalPlanner {

Ok(Arc::new(case_expr))
}
ExprStruct::ArrayRepeat(expr) => {
let src_expr =
self.create_expr(expr.left.as_ref().unwrap(), Arc::clone(&input_schema))?;
let count_expr =
self.create_expr(expr.right.as_ref().unwrap(), Arc::clone(&input_schema))?;
// Cast count_expr from Int32 to Int64 to support df count argument
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

apache/datafusion#14236

Currently we are manually casting the count_expr to Int64 datatype.
We can remove this once this PR is merged.

let count_expr: Arc<dyn PhysicalExpr> =
match count_expr.data_type(&input_schema.clone())? {
DataType::Int32 => Arc::new(CastExpr::new(
count_expr,
DataType::Int64,
Some(CastOptions::default()),
)),
_ => count_expr,
};

let args = vec![Arc::clone(&src_expr), Arc::clone(&count_expr)];

let datafusion_array_repeat = array_repeat_udf();
let data_types: Vec<DataType> = vec![
src_expr.data_type(&Arc::clone(&input_schema))?,
count_expr.data_type(&Arc::clone(&input_schema))?,
];
let return_type = datafusion_array_repeat.return_type(&data_types)?;

let array_repeat_expr: Arc<dyn PhysicalExpr> = Arc::new(ScalarFunctionExpr::new(
"array_repeat",
datafusion_array_repeat,
args,
return_type,
));

let is_null_expr: Arc<dyn PhysicalExpr> = Arc::new(IsNullExpr::new(count_expr));
let null_literal_expr: Arc<dyn PhysicalExpr> =
Arc::new(Literal::new(ScalarValue::Null));

let case_expr = CaseExpr::try_new(
None,
vec![(is_null_expr, null_literal_expr)],
Some(array_repeat_expr),
)?;
Ok(Arc::new(case_expr))
}
ExprStruct::ArrayIntersect(expr) => {
let left_expr =
self.create_expr(expr.left.as_ref().unwrap(), Arc::clone(&input_schema))?;
Expand Down
1 change: 1 addition & 0 deletions native/proto/src/proto/expr.proto
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,7 @@ message Expr {
BinaryExpr array_contains = 60;
BinaryExpr array_remove = 61;
BinaryExpr array_intersect = 62;
BinaryExpr array_repeat = 63;
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2302,6 +2302,12 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim
expr.children(1),
inputs,
(builder, binaryExpr) => builder.setArrayAppend(binaryExpr))
case expr if expr.prettyName == "array_repeat" =>
createBinaryExpr(
expr.children(0),
expr.children(1),
inputs,
(builder, binaryExpr) => builder.setArrayRepeat(binaryExpr))
case _ if expr.prettyName == "array_intersect" =>
createBinaryExpr(
expr.children(0),
Expand Down
16 changes: 16 additions & 0 deletions spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -2691,4 +2691,20 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper {
}
}

test("array_repeat") {
Seq(true, false).foreach { dictionaryEnabled =>
withTempDir { dir =>
val path = new Path(dir.toURI.toString, "test.parquet")
makeParquetFileAllTypes(path, dictionaryEnabled, 10000)
spark.read.parquet(path.toString).createOrReplaceTempView("t1")

checkSparkAnswerAndOperator(sql("SELECT array_repeat(_2, _4) from t1"))
checkSparkAnswerAndOperator(
sql("SELECT array_repeat(_2, 5) from t1 where _2 is not null"))
checkSparkAnswerAndOperator(sql("SELECT array_repeat(_3, 2) from t1 where _3 is null"))
checkSparkAnswerAndOperator(sql("SELECT array_repeat(cast(_3 as string), 2) from t1"))
checkSparkAnswerAndOperator(sql("SELECT array_repeat(array(_2, _3, _4), 2) from t1"))
}
}
}
}