From 768b3e90f261c7aea58bdb98dc698b90deeeae34 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Sun, 14 Dec 2025 16:24:01 +0400 Subject: [PATCH 1/4] impl map_from_entries --- native/core/src/execution/jni_api.rs | 2 + .../apache/comet/serde/QueryPlanSerde.scala | 3 +- .../scala/org/apache/comet/serde/maps.scala | 29 +++++++++++- .../comet/CometMapExpressionSuite.scala | 45 +++++++++++++++++++ 4 files changed, 77 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index a24d993059..4f53cea3e6 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -46,6 +46,7 @@ use datafusion_spark::function::datetime::date_add::SparkDateAdd; use datafusion_spark::function::datetime::date_sub::SparkDateSub; use datafusion_spark::function::hash::sha1::SparkSha1; use datafusion_spark::function::hash::sha2::SparkSha2; +use datafusion_spark::function::map::map_from_entries::MapFromEntries; use datafusion_spark::function::math::expm1::SparkExpm1; use datafusion_spark::function::string::char::CharFunc; use datafusion_spark::function::string::concat::SparkConcat; @@ -337,6 +338,7 @@ fn register_datafusion_spark_function(session_ctx: &SessionContext) { session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha1::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkConcat::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkBitwiseNot::default())); + session_ctx.register_udf(ScalarUDF::new_from_impl(MapFromEntries::default())); } /// Prepares arrow arrays for output. diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 54df2f1688..a99cf3824b 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -125,7 +125,8 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[MapKeys] -> CometMapKeys, classOf[MapEntries] -> CometMapEntries, classOf[MapValues] -> CometMapValues, - classOf[MapFromArrays] -> CometMapFromArrays) + classOf[MapFromArrays] -> CometMapFromArrays, + classOf[MapFromEntries] -> CometMapFromEntries) private val structExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( classOf[CreateNamedStruct] -> CometCreateNamedStruct, diff --git a/spark/src/main/scala/org/apache/comet/serde/maps.scala b/spark/src/main/scala/org/apache/comet/serde/maps.scala index 2e217f6af0..498aa3594c 100644 --- a/spark/src/main/scala/org/apache/comet/serde/maps.scala +++ b/spark/src/main/scala/org/apache/comet/serde/maps.scala @@ -19,9 +19,12 @@ package org.apache.comet.serde +import scala.annotation.tailrec + import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{ArrayType, MapType} +import org.apache.spark.sql.types.{ArrayType, BinaryType, DataType, MapType, StructType} +import org.apache.comet.serde.CometArrayReverse.containsBinary import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, optExprWithInfo, scalarFunctionExprToProto, scalarFunctionExprToProtoWithReturnType} object CometMapKeys extends CometExpressionSerde[MapKeys] { @@ -89,3 +92,27 @@ object CometMapFromArrays extends CometExpressionSerde[MapFromArrays] { optExprWithInfo(mapFromArraysExpr, expr, expr.children: _*) } } + +object CometMapFromEntries extends CometScalarFunction[MapFromEntries]("map_from_entries") { + val keyUnsupportedReason = "Using BinaryType as Map keys is not allowed in map_from_entries" + val valueUnsupportedReason = "Using BinaryType as Map values is not allowed in map_from_entries" + + private def containsBinary(dataType: DataType): Boolean = { + dataType match { + case BinaryType => true + case StructType(fields) => fields.exists(field => containsBinary(field.dataType)) + case ArrayType(elementType, _) => containsBinary(elementType) + case _ => false + } + } + + override def getSupportLevel(expr: MapFromEntries): SupportLevel = { + if (containsBinary(expr.dataType.keyType)) { + return Incompatible(Some(keyUnsupportedReason)) + } + if (containsBinary(expr.dataType.valueType)) { + return Incompatible(Some(valueUnsupportedReason)) + } + Compatible(None) + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala index 88c13391a6..01b9744ed6 100644 --- a/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala @@ -25,7 +25,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.BinaryType +import org.apache.comet.serde.CometMapFromEntries import org.apache.comet.testing.{DataGenOptions, ParquetGenerator, SchemaGenOptions} class CometMapExpressionSuite extends CometTestBase { @@ -125,4 +127,47 @@ class CometMapExpressionSuite extends CometTestBase { } } + test("map_from_entries") { + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + val filename = path.toString + val random = new Random(42) + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val schemaGenOptions = + SchemaGenOptions( + generateArray = true, + generateStruct = true, + primitiveTypes = SchemaGenOptions.defaultPrimitiveTypes.filterNot(_ == BinaryType)) + val dataGenOptions = DataGenOptions(allowNull = false, generateNegativeZero = false) + ParquetGenerator.makeParquetFile( + random, + spark, + filename, + 100, + schemaGenOptions, + dataGenOptions) + } + val df = spark.read.parquet(filename) + df.createOrReplaceTempView("t1") + for (field <- df.schema.fieldNames) { + checkSparkAnswerAndOperator( + spark.sql(s"SELECT map_from_entries(array(struct($field as a, $field as b))) FROM t1")) + } + } + } + + test("map_from_entries - fallback for binary type") { + val table = "t2" + withTable(table) { + sql( + s"create table $table using parquet as select cast(array() as array) as c1 from range(10)") + checkSparkAnswerAndFallbackReason( + sql(s"select map_from_entries(array(struct(c1, 0))) from $table"), + CometMapFromEntries.keyUnsupportedReason) + checkSparkAnswerAndFallbackReason( + sql(s"select map_from_entries(array(struct(0, c1))) from $table"), + CometMapFromEntries.valueUnsupportedReason) + } + } + } From c68c3428676b5d991e7ba9e13464bf2ce1ec84e8 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 16 Dec 2025 16:10:43 +0400 Subject: [PATCH 2/4] Revert "impl map_from_entries" This reverts commit 768b3e90f261c7aea58bdb98dc698b90deeeae34. --- native/core/src/execution/jni_api.rs | 2 - .../apache/comet/serde/QueryPlanSerde.scala | 3 +- .../scala/org/apache/comet/serde/maps.scala | 29 +----------- .../comet/CometMapExpressionSuite.scala | 45 ------------------- 4 files changed, 2 insertions(+), 77 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 4f53cea3e6..a24d993059 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -46,7 +46,6 @@ use datafusion_spark::function::datetime::date_add::SparkDateAdd; use datafusion_spark::function::datetime::date_sub::SparkDateSub; use datafusion_spark::function::hash::sha1::SparkSha1; use datafusion_spark::function::hash::sha2::SparkSha2; -use datafusion_spark::function::map::map_from_entries::MapFromEntries; use datafusion_spark::function::math::expm1::SparkExpm1; use datafusion_spark::function::string::char::CharFunc; use datafusion_spark::function::string::concat::SparkConcat; @@ -338,7 +337,6 @@ fn register_datafusion_spark_function(session_ctx: &SessionContext) { session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha1::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkConcat::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkBitwiseNot::default())); - session_ctx.register_udf(ScalarUDF::new_from_impl(MapFromEntries::default())); } /// Prepares arrow arrays for output. diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index a99cf3824b..54df2f1688 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -125,8 +125,7 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[MapKeys] -> CometMapKeys, classOf[MapEntries] -> CometMapEntries, classOf[MapValues] -> CometMapValues, - classOf[MapFromArrays] -> CometMapFromArrays, - classOf[MapFromEntries] -> CometMapFromEntries) + classOf[MapFromArrays] -> CometMapFromArrays) private val structExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( classOf[CreateNamedStruct] -> CometCreateNamedStruct, diff --git a/spark/src/main/scala/org/apache/comet/serde/maps.scala b/spark/src/main/scala/org/apache/comet/serde/maps.scala index 498aa3594c..2e217f6af0 100644 --- a/spark/src/main/scala/org/apache/comet/serde/maps.scala +++ b/spark/src/main/scala/org/apache/comet/serde/maps.scala @@ -19,12 +19,9 @@ package org.apache.comet.serde -import scala.annotation.tailrec - import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{ArrayType, BinaryType, DataType, MapType, StructType} +import org.apache.spark.sql.types.{ArrayType, MapType} -import org.apache.comet.serde.CometArrayReverse.containsBinary import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, optExprWithInfo, scalarFunctionExprToProto, scalarFunctionExprToProtoWithReturnType} object CometMapKeys extends CometExpressionSerde[MapKeys] { @@ -92,27 +89,3 @@ object CometMapFromArrays extends CometExpressionSerde[MapFromArrays] { optExprWithInfo(mapFromArraysExpr, expr, expr.children: _*) } } - -object CometMapFromEntries extends CometScalarFunction[MapFromEntries]("map_from_entries") { - val keyUnsupportedReason = "Using BinaryType as Map keys is not allowed in map_from_entries" - val valueUnsupportedReason = "Using BinaryType as Map values is not allowed in map_from_entries" - - private def containsBinary(dataType: DataType): Boolean = { - dataType match { - case BinaryType => true - case StructType(fields) => fields.exists(field => containsBinary(field.dataType)) - case ArrayType(elementType, _) => containsBinary(elementType) - case _ => false - } - } - - override def getSupportLevel(expr: MapFromEntries): SupportLevel = { - if (containsBinary(expr.dataType.keyType)) { - return Incompatible(Some(keyUnsupportedReason)) - } - if (containsBinary(expr.dataType.valueType)) { - return Incompatible(Some(valueUnsupportedReason)) - } - Compatible(None) - } -} diff --git a/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala index 01b9744ed6..88c13391a6 100644 --- a/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala @@ -25,9 +25,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.BinaryType -import org.apache.comet.serde.CometMapFromEntries import org.apache.comet.testing.{DataGenOptions, ParquetGenerator, SchemaGenOptions} class CometMapExpressionSuite extends CometTestBase { @@ -127,47 +125,4 @@ class CometMapExpressionSuite extends CometTestBase { } } - test("map_from_entries") { - withTempDir { dir => - val path = new Path(dir.toURI.toString, "test.parquet") - val filename = path.toString - val random = new Random(42) - withSQLConf(CometConf.COMET_ENABLED.key -> "false") { - val schemaGenOptions = - SchemaGenOptions( - generateArray = true, - generateStruct = true, - primitiveTypes = SchemaGenOptions.defaultPrimitiveTypes.filterNot(_ == BinaryType)) - val dataGenOptions = DataGenOptions(allowNull = false, generateNegativeZero = false) - ParquetGenerator.makeParquetFile( - random, - spark, - filename, - 100, - schemaGenOptions, - dataGenOptions) - } - val df = spark.read.parquet(filename) - df.createOrReplaceTempView("t1") - for (field <- df.schema.fieldNames) { - checkSparkAnswerAndOperator( - spark.sql(s"SELECT map_from_entries(array(struct($field as a, $field as b))) FROM t1")) - } - } - } - - test("map_from_entries - fallback for binary type") { - val table = "t2" - withTable(table) { - sql( - s"create table $table using parquet as select cast(array() as array) as c1 from range(10)") - checkSparkAnswerAndFallbackReason( - sql(s"select map_from_entries(array(struct(c1, 0))) from $table"), - CometMapFromEntries.keyUnsupportedReason) - checkSparkAnswerAndFallbackReason( - sql(s"select map_from_entries(array(struct(0, c1))) from $table"), - CometMapFromEntries.valueUnsupportedReason) - } - } - } From 21a577169f115d1dd8d751099bd60c52f2350cad Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 19 May 2026 10:06:28 +0400 Subject: [PATCH 3/4] WORK --- .../src/json_funcs/json_array_length.rs | 96 +++++++++++++++++++ native/spark-expr/src/json_funcs/mod.rs | 1 + 2 files changed, 97 insertions(+) create mode 100644 native/spark-expr/src/json_funcs/json_array_length.rs diff --git a/native/spark-expr/src/json_funcs/json_array_length.rs b/native/spark-expr/src/json_funcs/json_array_length.rs new file mode 100644 index 0000000000..8eb0e29c8d --- /dev/null +++ b/native/spark-expr/src/json_funcs/json_array_length.rs @@ -0,0 +1,96 @@ +// 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::any::Any; +use arrow::array::ArrayRef; +use arrow::datatypes::DataType; +use datafusion::common::{exec_err, Result, ScalarValue}; +use datafusion::common::cast::as_string_array; +use datafusion::logical_expr::{ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility}; + +#[derive(Debug, PartialEq, Eq, Hash)] +pub struct JsonArrayLength { + signature: Signature, +} + +impl Default for JsonArrayLength { + fn default() -> Self { + Self::new() + } +} + +impl JsonArrayLength { + pub fn new() -> Self { + Self { + signature: Signature::variadic(vec![DataType::Utf8], Volatility::Immutable), + } + } +} + +impl ScalarUDFImpl for JsonArrayLength { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "json_array_length" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Int32) + } + + fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { + spark_json_array_length(&args.args) + } +} + +fn spark_json_array_length(args: &[ColumnarValue]) -> Result { + if args.len() != 1 { + return exec_err!("json_array_length function takes exactly one argument"); + } + match &args[0] { + ColumnarValue::Array(array) => { + let result = spark_json_array_length_array(array)?; + Ok(ColumnarValue::Array(result)) + } + ColumnarValue::Scalar(scalar) => { + let result = spark_json_array_length_scalar(scalar)?; + Ok(ColumnarValue::Scalar(result)) + } + } +} + +fn spark_json_array_length_array(array: &ArrayRef) -> Result { + match array.data_type() { + DataType::Utf8 => { + let array = as_string_array(array)?; + Int32Array; + } + other => { + exec_err!("Unsupported data type {other:?} for function `json_array_length`") + } + } +} + +fn spark_json_array_length_scalar(scalar: &ScalarValue) -> Result { + unimplemented!() +} diff --git a/native/spark-expr/src/json_funcs/mod.rs b/native/spark-expr/src/json_funcs/mod.rs index 9f025070d7..eaa29f9bab 100644 --- a/native/spark-expr/src/json_funcs/mod.rs +++ b/native/spark-expr/src/json_funcs/mod.rs @@ -17,6 +17,7 @@ mod from_json; mod to_json; +mod json_array_length; pub use from_json::FromJson; pub use to_json::ToJson; From 57076f4fc972315db0403a0dc6876ea605bbac2d Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 19 May 2026 14:56:52 +0400 Subject: [PATCH 4/4] feat: impl json_array_length --- native/spark-expr/src/comet_scalar_funcs.rs | 2 + .../src/json_funcs/json_array_length.rs | 53 ++++++++++++++-- native/spark-expr/src/json_funcs/mod.rs | 3 +- .../apache/comet/serde/QueryPlanSerde.scala | 5 +- .../org/apache/comet/serde/statics.scala | 5 +- .../expressions/json/json_array_length.sql | 61 +++++++++++++++++++ 6 files changed, 120 insertions(+), 9 deletions(-) create mode 100644 spark/src/test/resources/sql-tests/expressions/json/json_array_length.sql diff --git a/native/spark-expr/src/comet_scalar_funcs.rs b/native/spark-expr/src/comet_scalar_funcs.rs index 9ecb11dc52..bddd439666 100644 --- a/native/spark-expr/src/comet_scalar_funcs.rs +++ b/native/spark-expr/src/comet_scalar_funcs.rs @@ -16,6 +16,7 @@ // under the License. use crate::hash_funcs::*; +use crate::json_funcs::JsonArrayLength; use crate::map_funcs::spark_map_sort; use crate::math_funcs::abs::abs; use crate::math_funcs::checked_arithmetic::{checked_add, checked_div, checked_mul, checked_sub}; @@ -216,6 +217,7 @@ fn all_scalar_functions() -> Vec> { Arc::new(ScalarUDF::new_from_impl(SparkMakeDate::default())), Arc::new(ScalarUDF::new_from_impl(SparkSecondsToTimestamp::default())), Arc::new(ScalarUDF::new_from_impl(SparkSizeFunc::default())), + Arc::new(ScalarUDF::new_from_impl(JsonArrayLength::default())), ] } diff --git a/native/spark-expr/src/json_funcs/json_array_length.rs b/native/spark-expr/src/json_funcs/json_array_length.rs index 8eb0e29c8d..289918363d 100644 --- a/native/spark-expr/src/json_funcs/json_array_length.rs +++ b/native/spark-expr/src/json_funcs/json_array_length.rs @@ -15,12 +15,16 @@ // specific language governing permissions and limitations // under the License. -use std::any::Any; -use arrow::array::ArrayRef; +use arrow::array::{Array, ArrayRef, Int32Builder}; use arrow::datatypes::DataType; -use datafusion::common::{exec_err, Result, ScalarValue}; use datafusion::common::cast::as_string_array; -use datafusion::logical_expr::{ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility}; +use datafusion::common::{exec_err, Result, ScalarValue}; +use datafusion::logical_expr::{ + ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, +}; + +use std::any::Any; +use std::sync::Arc; #[derive(Debug, PartialEq, Eq, Hash)] pub struct JsonArrayLength { @@ -83,7 +87,21 @@ fn spark_json_array_length_array(array: &ArrayRef) -> Result { match array.data_type() { DataType::Utf8 => { let array = as_string_array(array)?; - Int32Array; + let mut builder = Int32Builder::with_capacity(array.len()); + + for row_idx in 0..array.len() { + if array.is_null(row_idx) { + builder.append_null(); + } else { + let json_str = array.value(row_idx); + if let Some(json_array_length) = get_json_array_length(json_str) { + builder.append_value(json_array_length); + } else { + builder.append_null() + } + } + } + Ok(Arc::new(builder.finish())) } other => { exec_err!("Unsupported data type {other:?} for function `json_array_length`") @@ -92,5 +110,28 @@ fn spark_json_array_length_array(array: &ArrayRef) -> Result { } fn spark_json_array_length_scalar(scalar: &ScalarValue) -> Result { - unimplemented!() + match scalar { + ScalarValue::Utf8(value) => { + let length = value + .clone() + .and_then(|json_str| get_json_array_length(&json_str)); + Ok(ScalarValue::Int32(length)) + } + other => { + exec_err!("Unsupported data type {other:?} for function `json_array_length`") + } + } +} + +fn get_json_array_length(json_str: &str) -> Option { + match serde_json::from_str::(json_str) { + Ok(json_value) => { + if json_value.is_array() { + Some(json_value.as_array().unwrap().len() as i32) + } else { + None + } + } + Err(_) => None, + } } diff --git a/native/spark-expr/src/json_funcs/mod.rs b/native/spark-expr/src/json_funcs/mod.rs index eaa29f9bab..59e1e5dd58 100644 --- a/native/spark-expr/src/json_funcs/mod.rs +++ b/native/spark-expr/src/json_funcs/mod.rs @@ -16,8 +16,9 @@ // under the License. mod from_json; -mod to_json; mod json_array_length; +mod to_json; pub use from_json::FromJson; +pub use json_array_length::JsonArrayLength; pub use to_json::ToJson; diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index f9acad1d89..93f2250d77 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -247,6 +247,9 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { private val conversionExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( classOf[Cast] -> CometCast) + private val jsonExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( + classOf[LengthOfJsonArray] -> CometScalarFunction("json_array_length")) + private[comet] val miscExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( // TODO PromotePrecision classOf[Alias] -> CometAlias, @@ -271,7 +274,7 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { mathExpressions ++ hashExpressions ++ stringExpressions ++ conditionalExpressions ++ mapExpressions ++ predicateExpressions ++ structExpressions ++ bitwiseExpressions ++ miscExpressions ++ arrayExpressions ++ - temporalExpressions ++ conversionExpressions + temporalExpressions ++ conversionExpressions ++ jsonExpressions /** * Mapping of Spark aggregate expression class to Comet expression handler. diff --git a/spark/src/main/scala/org/apache/comet/serde/statics.scala b/spark/src/main/scala/org/apache/comet/serde/statics.scala index bff64e753a..9aa315764f 100644 --- a/spark/src/main/scala/org/apache/comet/serde/statics.scala +++ b/spark/src/main/scala/org/apache/comet/serde/statics.scala @@ -20,6 +20,7 @@ package org.apache.comet.serde import org.apache.spark.sql.catalyst.expressions.{Attribute, ExpressionImplUtils, Literal, UrlCodec} +import org.apache.spark.sql.catalyst.expressions.json.JsonExpressionUtils import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke import org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils @@ -38,7 +39,9 @@ object CometStaticInvoke extends CometExpressionSerde[StaticInvoke] { "read_side_padding"), ("isLuhnNumber", classOf[ExpressionImplUtils]) -> CometScalarFunction("luhn_check"), ("encode", UrlCodec.getClass) -> CometUrlEncodeStaticInvoke, - ("decode", UrlCodec.getClass) -> CometUrlDecodeStaticInvoke) + ("decode", UrlCodec.getClass) -> CometUrlDecodeStaticInvoke, + ("lengthOfJsonArray", classOf[JsonExpressionUtils]) -> CometScalarFunction( + "json_array_length")) override def convert( expr: StaticInvoke, diff --git a/spark/src/test/resources/sql-tests/expressions/json/json_array_length.sql b/spark/src/test/resources/sql-tests/expressions/json/json_array_length.sql new file mode 100644 index 0000000000..5d2e82f2cd --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/json/json_array_length.sql @@ -0,0 +1,61 @@ +-- 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. + +statement +CREATE TABLE test_json_array_length(j string) USING parquet + +statement +INSERT INTO test_json_array_length VALUES + ('[1,2,3,4]'), + ('[]'), + ('[1]'), + (NULL), + ('[1,2,3,{"f1":1,"f2":[5,6]},4]'), + ('[[1,2],[3,4],[5,6]]'), + ('[{"a":1},{"b":2},{"c":3}]'), + ('[1,2'), + ('[1,2,3,]'), + ('not a json'), + ('{"object": "not array"}'), + (''), + (' '), + ('[true, false, null]'), + ('["string1", "string2", "string3"]'), + ('[1, "mixed", true, null, {"key":"value"}]'), + ('[1,2,3,4,5,6,7,8,9,10]'), + ('["line1\nline2", "tab\tseparated", "quote\"here"]'), + ('{"outer": [1,2,3], "inner": [[1,2],[3,4]]}'), + ('{"arrays": {"first": [1,2], "second": [3,4,5]}}'), + ('[{"arr": [1,2,3]}, {"arr": [4,5]}]') + +query +SELECT json_array_length(j) FROM test_json_array_length + +query +SELECT json_array_length('[1,2,3,4]') + +query +SELECT json_array_length('not an array') + +query +SELECT json_array_length('{"key":"value"}') + +query +SELECT json_array_length(NULL) + +query +SELECT json_array_length('[]')