From 383ae9197422da5948a31caf21722569ef2067fe Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 20 May 2024 08:31:07 -0600 Subject: [PATCH 01/14] chore: improve fallback message when comet native shuffle is not enabled (#445) * improve fallback message when comet native shuffle is not enabled * update test --- .../comet/CometSparkSessionExtensions.scala | 32 +++++++++++++++---- .../apache/comet/CometExpressionSuite.scala | 6 ++-- .../org/apache/spark/sql/CometTestBase.scala | 5 ++- 3 files changed, 33 insertions(+), 10 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 7c269c411..85a19f55c 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -30,8 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{Final, Partial} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.comet._ -import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometNativeShuffle} -import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec +import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometNativeShuffle, CometShuffleExchangeExec, CometShuffleManager} import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec} @@ -46,7 +45,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.comet.CometConf._ -import org.apache.comet.CometSparkSessionExtensions.{createMessage, isANSIEnabled, isCometBroadCastForceEnabled, isCometColumnarShuffleEnabled, isCometEnabled, isCometExecEnabled, isCometOperatorEnabled, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSchemaSupported, isSpark34Plus, shouldApplyRowToColumnar, withInfo, withInfos} +import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometColumnarShuffleEnabled, isCometEnabled, isCometExecEnabled, isCometOperatorEnabled, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSchemaSupported, isSpark34Plus, shouldApplyRowToColumnar, withInfo, withInfos} import org.apache.comet.parquet.{CometParquetScan, SupportsComet} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde @@ -684,7 +683,8 @@ class CometSparkSessionExtensions case s: ShuffleExchangeExec => val isShuffleEnabled = isCometShuffleEnabled(conf) - val msg1 = createMessage(!isShuffleEnabled, "Native shuffle is not enabled") + val reason = getCometShuffleNotEnabledReason(conf).getOrElse("no reason available") + val msg1 = createMessage(!isShuffleEnabled, s"Native shuffle is not enabled: $reason") val columnarShuffleEnabled = isCometColumnarShuffleEnabled(conf) val msg2 = createMessage( isShuffleEnabled && !columnarShuffleEnabled && !QueryPlanSerde @@ -933,13 +933,31 @@ object CometSparkSessionExtensions extends Logging { } private[comet] def isCometShuffleEnabled(conf: SQLConf): Boolean = - COMET_EXEC_SHUFFLE_ENABLED.get(conf) && - (conf.contains("spark.shuffle.manager") && conf.getConfString("spark.shuffle.manager") == - "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") && + COMET_EXEC_SHUFFLE_ENABLED.get(conf) && isCometShuffleManagerEnabled(conf) && // TODO: AQE coalesce partitions feature causes Comet shuffle memory leak. // We should disable Comet shuffle when AQE coalesce partitions is enabled. (!conf.coalesceShufflePartitionsEnabled || COMET_SHUFFLE_ENFORCE_MODE_ENABLED.get()) + private[comet] def getCometShuffleNotEnabledReason(conf: SQLConf): Option[String] = { + if (!COMET_EXEC_SHUFFLE_ENABLED.get(conf)) { + Some(s"${COMET_EXEC_SHUFFLE_ENABLED.key} is not enabled") + } else if (!isCometShuffleManagerEnabled(conf)) { + Some(s"spark.shuffle.manager is not set to ${CometShuffleManager.getClass.getName}") + } else if (conf.coalesceShufflePartitionsEnabled && !COMET_SHUFFLE_ENFORCE_MODE_ENABLED + .get()) { + Some( + s"${SQLConf.COALESCE_PARTITIONS_ENABLED.key} is enabled and " + + s"${COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key} is not enabled") + } else { + None + } + } + + private def isCometShuffleManagerEnabled(conf: SQLConf) = { + conf.contains("spark.shuffle.manager") && conf.getConfString("spark.shuffle.manager") == + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager" + } + private[comet] def isCometScanEnabled(conf: SQLConf): Boolean = { COMET_SCAN_ENABLED.get(conf) } diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index f3fd50e9e..98a2bad02 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -1420,14 +1420,16 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { "extractintervalmonths is not supported")), ( s"SELECT sum(c0), sum(c2) from $table group by c1", - Set("Native shuffle is not enabled", "AQEShuffleRead is not supported")), + Set( + "Native shuffle is not enabled: spark.comet.exec.shuffle.enabled is not enabled", + "AQEShuffleRead is not supported")), ( "SELECT A.c1, A.sum_c0, A.sum_c2, B.casted from " + s"(SELECT c1, sum(c0) as sum_c0, sum(c2) as sum_c2 from $table group by c1) as A, " + s"(SELECT c1, cast(make_interval(c0, c1, c0, c1, c0, c0, c2) as string) as casted from $table) as B " + "where A.c1 = B.c1 ", Set( - "Native shuffle is not enabled", + "Native shuffle is not enabled: spark.comet.exec.shuffle.enabled is not enabled", "AQEShuffleRead is not supported", "make_interval is not supported", "BroadcastExchange is not supported", diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala index 112d35b13..0530d764c 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -261,7 +261,10 @@ abstract class CometTestBase } val extendedInfo = new ExtendedExplainInfo().generateExtendedInfo(dfComet.queryExecution.executedPlan) - assert(extendedInfo.equalsIgnoreCase(expectedInfo.toSeq.sorted.mkString("\n"))) + val expectedStr = expectedInfo.toSeq.sorted.mkString("\n") + if (!extendedInfo.equalsIgnoreCase(expectedStr)) { + fail(s"$extendedInfo != $expectedStr (case-insensitive comparison)") + } } private var _spark: SparkSession = _ From 9acb3df5161e590c7c262cfaf2a35a80177bc505 Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Mon, 20 May 2024 13:41:22 -0700 Subject: [PATCH 02/14] docs: add guide to adding a new expression (#422) * docs: add guide to adding a new expression * docs: revise with presentation info * docs: fix warning * docs: fix header level * docs: better info about handling datafusion udfs * docs: grammar/typos/etc * docs: clarify datafusion/datafusion comet path * docs: clarify language about `isSpark32` * docs: fix error --- .../adding_a_new_expression.md | 212 ++++++++++++++++++ docs/source/index.rst | 1 + 2 files changed, 213 insertions(+) create mode 100644 docs/source/contributor-guide/adding_a_new_expression.md diff --git a/docs/source/contributor-guide/adding_a_new_expression.md b/docs/source/contributor-guide/adding_a_new_expression.md new file mode 100644 index 000000000..6cf10c758 --- /dev/null +++ b/docs/source/contributor-guide/adding_a_new_expression.md @@ -0,0 +1,212 @@ + + +# Adding a New Expression + +There are a number of Spark expression that are not supported by DataFusion Comet yet, and implementing them is a good way to contribute to the project. + +Before you start, have a look through [these slides](https://docs.google.com/presentation/d/1H0fF2MOkkBK8fPBlnqK6LejUeLcVD917JhVWfp3mb8A/edit#slide=id.p) as they provide a conceptual overview. And a video of a presentation on those slides is available [here](https://drive.google.com/file/d/1POU4lFAZfYwZR8zV1X2eoLiAmc1GDtSP/view?usp=sharing). + +## Finding an Expression to Add + +You may have a specific expression in mind that you'd like to add, but if not, you can review the [expression coverage document](https://github.com/apache/datafusion-comet/blob/f08fcadd5fbdb5b04293d33e654f6c16f81b70c4/doc/spark_builtin_expr_coverage.txt) to see which expressions are not yet supported. + +## Implementing the Expression + +Once you have the expression you'd like to add, you should take inventory of the following: + +1. What is the Spark expression's behavior across different Spark versions? These make good test cases and will inform you of any compatibility issues, such as an API change that will have to be addressed. +2. Check if the expression is already implemented in DataFusion and if it is compatible with the Spark expression. + 1. If it is, you can potentially reuse the existing implementation though you'll need to add tests to verify compatibility. + 2. If it's not, consider an initial version in DataFusion for expressions that are common across different engines. For expressions that are specific to Spark, consider an initial version in DataFusion Comet. +3. Test cases for the expression. As mentioned, you can refer to Spark's test cases for a good idea of what to test. + +Once you know what you want to add, you'll need to update the query planner to recognize the new expression in Scala and potentially add a new expression implementation in the Rust package. + +### Adding the Expression in Scala + +The `QueryPlanSerde` object has a method `exprToProto`, which is responsible for converting a Spark expression to a protobuf expression. Within that method is an `exprToProtoInternal` method that contains a large match statement for each expression type. You'll need to add a new case to this match statement for your new expression. + +For example, the `unhex` function looks like this: + +```scala +case e: Unhex if !isSpark32 => + val unHex = unhexSerde(e) + + val childExpr = exprToProtoInternal(unHex._1, inputs) + val failOnErrorExpr = exprToProtoInternal(unHex._2, inputs) + + val optExpr = + scalarExprToProtoWithReturnType("unhex", e.dataType, childExpr, failOnErrorExpr) + optExprWithInfo(optExpr, expr, unHex._1) +``` + +A few things to note here: + +* The `isSpark32` check is used to fall back to Spark's implementation of `unhex` in Spark 3.2. This is somewhat context specific, because in this case, due to a bug in Spark 3.2 for `unhex`, we want to use the Spark implementation and not a Comet implementation that would behave differently if correct. +* The function is recursively called on child expressions, so you'll need to make sure that the child expressions are also converted to protobuf. +* `scalarExprToProtoWithReturnType` is for scalar functions that need return type information. Your expression may use a different method depending on the type of expression. + +#### Adding Spark-side Tests for the New Expression + +It is important to verify that the new expression is correctly recognized by the native execution engine and matches the expected spark behavior. To do this, you can add a set of test cases in the `CometExpressionSuite`, and use the `checkSparkAnswerAndOperator` method to compare the results of the new expression with the expected Spark results and that Comet's native execution engine is able to execute the expression. + +For example, this is the test case for the `unhex` expression: + +```scala +test("unhex") { + assume(!isSpark32, "unhex function has incorrect behavior in 3.2") // used to skip the test in Spark 3.2 + + val table = "unhex_table" + withTable(table) { + sql(s"create table $table(col string) using parquet") + + sql(s"""INSERT INTO $table VALUES + |('537061726B2053514C'), + |('737472696E67'), + |('\\0'), + |(''), + |('###'), + |('G123'), + |('hello'), + |('A1B'), + |('0A1B')""".stripMargin) + + checkSparkAnswerAndOperator(s"SELECT unhex(col) FROM $table") + } +} +``` + +### Adding the Expression To the Protobuf Definition + +Once you have the expression implemented in Scala, you might need to update the protobuf definition to include the new expression. You may not need to do this if the expression is already covered by the existing protobuf definition (e.g. you're adding a new scalar function). + +You can find the protobuf definition in `expr.proto`, and in particular the `Expr` or potentially the `AggExpr`. These are similar in theory to the large case statement in `QueryPlanSerde`, but in protobuf format. So if you were to add a new expression called `Add2`, you would add a new case to the `Expr` message like so: + +```proto +message Expr { + oneof expr_struct { + ... + Add2 add2 = 100; // Choose the next available number + } +} +``` + +Then you would define the `Add2` message like so: + +```proto +message Add2 { + Expr left = 1; + Expr right = 2; +} +``` + +### Adding the Expression in Rust + +With the serialization complete, the next step is to implement the expression in Rust and ensure that the incoming plan can make use of it. + +How this works, is somewhat dependent on the type of expression you're adding, so see the `core/src/execution/datafusion/expressions` directory for examples of how to implement different types of expressions. + +#### Generally Adding a New Expression + +If you're adding a new expression, you'll need to review `create_plan` and `create_expr`. `create_plan` is responsible for translating the incoming plan into a DataFusion plan, and may delegate to `create_expr` to create the physical expressions for the plan. + +If you added a new message to the protobuf definition, you'll add a new match case to the `create_expr` method to handle the new expression. For example, if you added an `Add2` expression, you would add a new case like so: + +```rust +match spark_expr.expr_struct.as_ref().unwrap() { + ... + ExprStruct::Add2(add2) => self.create_binary_expr(...) +} +``` + +`self.create_binary_expr` is for a binary expression, but if something out of the box is needed, you can create a new `PhysicalExpr` implementation. For example, see `if_expr.rs` for an example of an implementation that doesn't fit the `create_binary_expr` mold. + +#### Adding a New Scalar Function Expression + +For a new scalar function, you can reuse a lot of code by updating the `create_comet_physical_fun` method to match on the function name and make the scalar UDF to be called. For example, the diff to add the `unhex` function is: + +```diff +macro_rules! make_comet_scalar_udf { + ($name:expr, $func:ident, $data_type:ident) => {{ + ++ "unhex" => { ++ let func = Arc::new(spark_unhex); ++ make_comet_scalar_udf!("unhex", func, without data_type) ++ } + + }} +} +``` + +With that addition, you can now implement the spark function in Rust. This function will look very similar to DataFusion code. For examples, see the `core/src/execution/datafusion/expressions/scalar_funcs` directory. + +Without getting into the internals, the function signature will look like: + +```rust +pub(super) fn spark_unhex(args: &[ColumnarValue]) -> Result { + // Do the work here +} +``` + +> **_NOTE:_** If you call the `make_comet_scalar_udf` macro with the data type, the function signature will look include the data type as a second argument. + +### API Differences Between Spark Versions + +If the expression you're adding has different behavior across different Spark versions, you'll need to account for that in your implementation. There are two tools at your disposal to help with this: + +1. Shims that exist in `spark/src/main/spark-$SPARK_VERSION/org/apache/comet/shims/CometExprShim.scala` for each Spark version. These shims are used to provide compatibility between different Spark versions. +2. Variables that correspond to the Spark version, such as `isSpark32`, which can be used to conditionally execute code based on the Spark version. + +## Shimming to Support Different Spark Versions + +By adding shims for each Spark version, you can provide a consistent interface for the expression across different Spark versions. For example, `unhex` added a new optional parameter is Spark 3.4, for if it should `failOnError` or not. So for version 3.2 and 3.3, the shim is: + +```scala +trait CometExprShim { + /** + * Returns a tuple of expressions for the `unhex` function. + */ + def unhexSerde(unhex: Unhex): (Expression, Expression) = { + (unhex.child, Literal(false)) + } +} +``` + +And for version 3.4, the shim is: + +```scala +trait CometExprShim { + /** + * Returns a tuple of expressions for the `unhex` function. + */ + def unhexSerde(unhex: Unhex): (Expression, Expression) = { + (unhex.child, unhex.failOnError) + } +} +``` + +Then when `unhexSerde` is called in the `QueryPlanSerde` object, it will use the correct shim for the Spark version. + +## Resources + +* [Variance PR](https://github.com/apache/datafusion-comet/pull/297) + * Aggregation function +* [Unhex PR](https://github.com/apache/datafusion-comet/pull/342) + * Basic scalar function with shims for different Spark versions diff --git a/docs/source/index.rst b/docs/source/index.rst index 819f72014..1e9eba21f 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -59,6 +59,7 @@ as a native runtime to achieve improvement in terms of query efficiency and quer Development Guide Debugging Guide Benchmarking Guide + Adding a New Expression Profiling Native Code Github and Issue Tracker From 9ca63a23edf67033e4f4eba5a9d004aa472743d2 Mon Sep 17 00:00:00 2001 From: Oleks V Date: Mon, 20 May 2024 18:03:23 -0700 Subject: [PATCH 03/14] Coverage: Add a manual test to show what Spark built in expression the DF can support directly (#331) * Coverage: Add a manual test for DF supporting Spark expressions directly Co-authored-by: advancedxy --- doc/spark_builtin_expr_coverage.txt | 419 ------------------ doc/spark_builtin_expr_coverage_agg.txt | 9 - docs/spark_builtin_expr_coverage.txt | 419 ++++++++++++++++++ pom.xml | 2 +- .../comet/CometExpressionCoverageSuite.scala | 143 ++++-- 5 files changed, 532 insertions(+), 460 deletions(-) delete mode 100644 doc/spark_builtin_expr_coverage.txt delete mode 100644 doc/spark_builtin_expr_coverage_agg.txt create mode 100644 docs/spark_builtin_expr_coverage.txt diff --git a/doc/spark_builtin_expr_coverage.txt b/doc/spark_builtin_expr_coverage.txt deleted file mode 100644 index 9083a068c..000000000 --- a/doc/spark_builtin_expr_coverage.txt +++ /dev/null @@ -1,419 +0,0 @@ -+---------------------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -|name |details | -+---------------------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -|! |{PASSED, [{SELECT ! true;, OK}]} | -|% |{PASSED, [{SELECT 2 % 1.8;, OK}]} | -|& |{PASSED, [{SELECT 3 & 5;, OK}]} | -|* |{PASSED, [{SELECT 2 * 3;, OK}]} | -|+ |{PASSED, [{SELECT 1 + 2;, OK}]} | -|- |{PASSED, [{SELECT 2 - 1;, OK}]} | -|/ |{PASSED, [{SELECT 3 / 2;, OK}]} | -|< |{PASSED, [{SELECT 1 < 2;, OK}]} | -|<= |{PASSED, [{SELECT 2 <= 2;, OK}]} | -|<=> |{PASSED, [{SELECT 2 <=> 2;, OK}]} | -|= |{PASSED, [{SELECT 2 = 2;, OK}]} | -|== |{PASSED, [{SELECT 2 == 2;, OK}]} | -|> |{PASSED, [{SELECT 2 > 1;, OK}]} | -|>= |{PASSED, [{SELECT 2 >= 1;, OK}]} | -|^ |{PASSED, [{SELECT 3 ^ 5;, OK}]} | -|abs |{PASSED, [{SELECT abs(-1);, OK}]} | -|acos |{PASSED, [{SELECT acos(1);, OK}]} | -|acosh |{FAILED, [{SELECT acosh(1);, Unsupported}]} | -|add_months |{FAILED, [{SELECT add_months('2016-08-31', 1);, Unsupported}]} | -|aes_decrypt |{FAILED, [{SELECT aes_decrypt(unhex('83F16B2AA704794132802D248E6BFD4E380078182D1544813898AC97E709B28A94'), '0000111122223333');, Unsupported}]} | -|aes_encrypt |{FAILED, [{SELECT hex(aes_encrypt('Spark', '0000111122223333'));, Unsupported}]} | -|aggregate |{FAILED, [{SELECT aggregate(array(1, 2, 3), 0, (acc, x) -> acc + x);, Unsupported}]} | -|and |{PASSED, [{SELECT true and true;, OK}]} | -|any |{FAILED, [{SELECT any(col) FROM VALUES (true), (false), (false) AS tab(col);, Unsupported}]} | -|any_value |{FAILED, [{SELECT any_value(col) FROM VALUES (10), (5), (20) AS tab(col);, Unsupported}]} | -|approx_count_distinct |{FAILED, [{SELECT approx_count_distinct(col1) FROM VALUES (1), (1), (2), (2), (3) tab(col1);, Unsupported}]} | -|approx_percentile |{FAILED, [{SELECT approx_percentile(col, array(0.5, 0.4, 0.1), 100) FROM VALUES (0), (1), (2), (10) AS tab(col);, Unsupported}]} | -|array |{FAILED, [{SELECT array(1, 2, 3);, Unsupported}]} | -|array_agg |{FAILED, [{SELECT array_agg(col) FROM VALUES (1), (2), (1) AS tab(col);, Unsupported}]} | -|array_append |{FAILED, [{SELECT array_append(array('b', 'd', 'c', 'a'), 'd');, Unsupported}]} | -|array_compact |{FAILED, [{SELECT array_compact(array(1, 2, 3, null));, Unsupported}]} | -|array_contains |{FAILED, [{SELECT array_contains(array(1, 2, 3), 2);, Unsupported}]} | -|array_distinct |{FAILED, [{SELECT array_distinct(array(1, 2, 3, null, 3));, Unsupported}]} | -|array_except |{FAILED, [{SELECT array_except(array(1, 2, 3), array(1, 3, 5));, Unsupported}]} | -|array_insert |{FAILED, [{SELECT array_insert(array(1, 2, 3, 4), 5, 5);, Unsupported}]} | -|array_intersect |{FAILED, [{SELECT array_intersect(array(1, 2, 3), array(1, 3, 5));, Unsupported}]} | -|array_join |{FAILED, [{SELECT array_join(array('hello', 'world'), ' ');, Unsupported}]} | -|array_max |{FAILED, [{SELECT array_max(array(1, 20, null, 3));, Unsupported}]} | -|array_min |{FAILED, [{SELECT array_min(array(1, 20, null, 3));, Unsupported}]} | -|array_position |{FAILED, [{SELECT array_position(array(3, 2, 1), 1);, Unsupported}]} | -|array_remove |{FAILED, [{SELECT array_remove(array(1, 2, 3, null, 3), 3);, Unsupported}]} | -|array_repeat |{FAILED, [{SELECT array_repeat('123', 2);, Unsupported}]} | -|array_size |{FAILED, [{SELECT array_size(array('b', 'd', 'c', 'a'));, Unsupported}]} | -|array_sort |{FAILED, [{SELECT array_sort(array(5, 6, 1), (left, right) -> case when left < right then -1 when left > right then 1 else 0 end);, Unsupported}]} | -|array_union |{FAILED, [{SELECT array_union(array(1, 2, 3), array(1, 3, 5));, Unsupported}]} | -|arrays_overlap |{FAILED, [{SELECT arrays_overlap(array(1, 2, 3), array(3, 4, 5));, Unsupported}]} | -|arrays_zip |{FAILED, [{SELECT arrays_zip(array(1, 2, 3), array(2, 3, 4));, Unsupported}]} | -|ascii |{PASSED, [{SELECT ascii('222');, OK}]} | -|asin |{PASSED, [{SELECT asin(0);, OK}]} | -|asinh |{FAILED, [{SELECT asinh(0);, Unsupported}]} | -|assert_true |{FAILED, [{SELECT assert_true(0 < 1);, Unsupported}]} | -|atan |{PASSED, [{SELECT atan(0);, OK}]} | -|atan2 |{PASSED, [{SELECT atan2(0, 0);, OK}]} | -|atanh |{FAILED, [{SELECT atanh(0);, Unsupported}]} | -|avg |{FAILED, [{SELECT avg(col) FROM VALUES (1), (2), (3) AS tab(col);, Unsupported}]} | -|base64 |{FAILED, [{SELECT base64('Spark SQL');, Unsupported}]} | -|bigint |{SKIPPED, [{, No examples found in spark.sessionState.functionRegistry}]} | -|bin |{FAILED, [{SELECT bin(13);, Unsupported}]} | -|binary |{SKIPPED, [{, No examples found in spark.sessionState.functionRegistry}]} | -|bit_and |{FAILED, [{SELECT bit_and(col) FROM VALUES (3), (5) AS tab(col);, Unsupported}]} | -|bit_count |{FAILED, [{SELECT bit_count(0);, Unsupported}]} | -|bit_get |{FAILED, [{SELECT bit_get(11, 0);, Unsupported}]} | -|bit_length |{PASSED, [{SELECT bit_length('Spark SQL');, OK}]} | -|bit_or |{FAILED, [{SELECT bit_or(col) FROM VALUES (3), (5) AS tab(col);, Unsupported}]} | -|bit_xor |{FAILED, [{SELECT bit_xor(col) FROM VALUES (3), (5) AS tab(col);, Unsupported}]} | -|bool_and |{FAILED, [{SELECT bool_and(col) FROM VALUES (true), (true), (true) AS tab(col);, Unsupported}]} | -|bool_or |{FAILED, [{SELECT bool_or(col) FROM VALUES (true), (false), (false) AS tab(col);, Unsupported}]} | -|boolean |{SKIPPED, [{, No examples found in spark.sessionState.functionRegistry}]} | -|bround |{FAILED, [{SELECT bround(2.5, 0);, Unsupported}]} | -|btrim |{PASSED, [{SELECT btrim(' SparkSQL ');, OK}]} | -|cardinality |{FAILED, [{SELECT cardinality(array('b', 'd', 'c', 'a'));, Unsupported}]} | -|cast |{PASSED, [{SELECT cast('10' as int);, OK}]} | -|cbrt |{FAILED, [{SELECT cbrt(27.0);, Unsupported}]} | -|ceil |{PASSED, [{SELECT ceil(-0.1);, OK}]} | -|ceiling |{PASSED, [{SELECT ceiling(-0.1);, OK}]} | -|char |{PASSED, [{SELECT char(65);, OK}]} | -|char_length |{PASSED, [{SELECT char_length('Spark SQL ');, OK}]} | -|character_length |{PASSED, [{SELECT character_length('Spark SQL ');, OK}]} | -|chr |{PASSED, [{SELECT chr(65);, OK}]} | -|coalesce |{PASSED, [{SELECT coalesce(NULL, 1, NULL);, OK}]} | -|collect_list |{FAILED, [{SELECT collect_list(col) FROM VALUES (1), (2), (1) AS tab(col);, Unsupported}]} | -|collect_set |{FAILED, [{SELECT collect_set(col) FROM VALUES (1), (2), (1) AS tab(col);, Unsupported}]} | -|concat |{FAILED, [{SELECT concat('Spark', 'SQL');, Unsupported}]} | -|concat_ws |{PASSED, [{SELECT concat_ws(' ', 'Spark', 'SQL');, OK}]} | -|contains |{FAILED, [{SELECT contains('Spark SQL', 'Spark');, Failed on native side}]} | -|conv |{FAILED, [{SELECT conv('100', 2, 10);, Unsupported}]} | -|convert_timezone |{FAILED, [{SELECT convert_timezone('Europe/Brussels', 'America/Los_Angeles', timestamp_ntz'2021-12-06 00:00:00');, Unsupported}]} | -|corr |{FAILED, [{SELECT corr(c1, c2) FROM VALUES (3, 2), (3, 3), (6, 4) as tab(c1, c2);, Unsupported}]} | -|cos |{PASSED, [{SELECT cos(0);, OK}]} | -|cosh |{FAILED, [{SELECT cosh(0);, Unsupported}]} | -|cot |{FAILED, [{SELECT cot(1);, Unsupported}]} | -|count |{FAILED, [{SELECT count(*) FROM VALUES (NULL), (5), (5), (20) AS tab(col);, Unsupported}]} | -|count_if |{FAILED, [{SELECT count_if(col % 2 = 0) FROM VALUES (NULL), (0), (1), (2), (3) AS tab(col);, Unsupported}]} | -|count_min_sketch |{FAILED, [{SELECT hex(count_min_sketch(col, 0.5d, 0.5d, 1)) FROM VALUES (1), (2), (1) AS tab(col);, Unsupported}]} | -|covar_pop |{FAILED, [{SELECT covar_pop(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2);, Unsupported}]} | -|covar_samp |{FAILED, [{SELECT covar_samp(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2);, Unsupported}]} | -|crc32 |{FAILED, [{SELECT crc32('Spark');, Unsupported}]} | -|csc |{FAILED, [{SELECT csc(1);, Unsupported}]} | -|cume_dist |{FAILED, [{SELECT a, b, cume_dist() OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b);, Unsupported}]} | -|curdate |{PASSED, [{SELECT curdate();, OK}]} | -|current_catalog |{PASSED, [{SELECT current_catalog();, OK}]} | -|current_database |{PASSED, [{SELECT current_database();, OK}]} | -|current_date |{PASSED, [{SELECT current_date();, OK}]} | -|current_schema |{PASSED, [{SELECT current_schema();, OK}]} | -|current_timestamp |{FAILED, [{SELECT current_timestamp();, Failed on something else. Check query manually}]} | -|current_timezone |{PASSED, [{SELECT current_timezone();, OK}]} | -|current_user |{PASSED, [{SELECT current_user();, OK}]} | -|date |{SKIPPED, [{, No examples found in spark.sessionState.functionRegistry}]} | -|date_add |{FAILED, [{SELECT date_add('2016-07-30', 1);, Unsupported}]} | -|date_diff |{FAILED, [{SELECT date_diff('2009-07-31', '2009-07-30');, Unsupported}]} | -|date_format |{FAILED, [{SELECT date_format('2016-04-08', 'y');, Unsupported}]} | -|date_from_unix_date |{FAILED, [{SELECT date_from_unix_date(1);, Unsupported}]} | -|date_part |{PASSED, [{SELECT date_part('YEAR', TIMESTAMP '2019-08-12 01:00:00.123456');, OK}]} | -|date_sub |{FAILED, [{SELECT date_sub('2016-07-30', 1);, Unsupported}]} | -|date_trunc |{FAILED, [{SELECT date_trunc('YEAR', '2015-03-05T09:32:05.359');, Failed on native side}]} | -|dateadd |{FAILED, [{SELECT dateadd('2016-07-30', 1);, Unsupported}]} | -|datediff |{FAILED, [{SELECT datediff('2009-07-31', '2009-07-30');, Unsupported}]} | -|datepart |{PASSED, [{SELECT datepart('YEAR', TIMESTAMP '2019-08-12 01:00:00.123456');, OK}]} | -|day |{FAILED, [{SELECT day('2009-07-30');, Unsupported}]} | -|dayofmonth |{FAILED, [{SELECT dayofmonth('2009-07-30');, Unsupported}]} | -|dayofweek |{FAILED, [{SELECT dayofweek('2009-07-30');, Unsupported}]} | -|dayofyear |{FAILED, [{SELECT dayofyear('2016-04-09');, Unsupported}]} | -|decimal |{SKIPPED, [{, No examples found in spark.sessionState.functionRegistry}]} | -|decode |{FAILED, [{SELECT decode(encode('abc', 'utf-8'), 'utf-8');, Unsupported}]} | -|degrees |{FAILED, [{SELECT degrees(3.141592653589793);, Unsupported}]} | -|dense_rank |{FAILED, [{SELECT a, b, dense_rank(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b);, Unsupported}]} | -|div |{FAILED, [{SELECT 3 div 2;, Unsupported}]} | -|double |{SKIPPED, [{, No examples found in spark.sessionState.functionRegistry}]} | -|e |{FAILED, [{SELECT e();, Unsupported}]} | -|element_at |{FAILED, [{SELECT element_at(array(1, 2, 3), 2);, Unsupported}]} | -|elt |{FAILED, [{SELECT elt(1, 'scala', 'java');, Unsupported}]} | -|encode |{FAILED, [{SELECT encode('abc', 'utf-8');, Unsupported}]} | -|endswith |{FAILED, [{SELECT endswith('Spark SQL', 'SQL');, Failed on native side}]} | -|equal_null |{PASSED, [{SELECT equal_null(3, 3);, OK}]} | -|every |{FAILED, [{SELECT every(col) FROM VALUES (true), (true), (true) AS tab(col);, Unsupported}]} | -|exists |{FAILED, [{SELECT exists(array(1, 2, 3), x -> x % 2 == 0);, Unsupported}]} | -|exp |{PASSED, [{SELECT exp(0);, OK}]} | -|explode |{FAILED, [{SELECT explode(array(10, 20));, Unsupported}]} | -|explode_outer |{FAILED, [{SELECT explode_outer(array(10, 20));, Unsupported}]} | -|expm1 |{FAILED, [{SELECT expm1(0);, Unsupported}]} | -|extract |{PASSED, [{SELECT extract(YEAR FROM TIMESTAMP '2019-08-12 01:00:00.123456');, OK}]} | -|factorial |{FAILED, [{SELECT factorial(5);, Unsupported}]} | -|filter |{FAILED, [{SELECT filter(array(1, 2, 3), x -> x % 2 == 1);, Unsupported}]} | -|find_in_set |{FAILED, [{SELECT find_in_set('ab','abc,b,ab,c,def');, Unsupported}]} | -|first |{FAILED, [{SELECT first(col) FROM VALUES (10), (5), (20) AS tab(col);, Unsupported}]} | -|first_value |{FAILED, [{SELECT first_value(col) FROM VALUES (10), (5), (20) AS tab(col);, Unsupported}]} | -|flatten |{FAILED, [{SELECT flatten(array(array(1, 2), array(3, 4)));, Unsupported}]} | -|float |{SKIPPED, [{, No examples found in spark.sessionState.functionRegistry}]} | -|floor |{PASSED, [{SELECT floor(-0.1);, OK}]} | -|forall |{FAILED, [{SELECT forall(array(1, 2, 3), x -> x % 2 == 0);, Unsupported}]} | -|format_number |{FAILED, [{SELECT format_number(12332.123456, 4);, Unsupported}]} | -|format_string |{FAILED, [{SELECT format_string("Hello World %d %s", 100, "days");, Unsupported}]} | -|from_csv |{FAILED, [{SELECT from_csv('1, 0.8', 'a INT, b DOUBLE');, Unsupported}]} | -|from_json |{FAILED, [{SELECT from_json('{"a":1, "b":0.8}', 'a INT, b DOUBLE');, Unsupported}]} | -|from_unixtime |{FAILED, [{SELECT from_unixtime(0, 'yyyy-MM-dd HH:mm:ss');, Unsupported}]} | -|from_utc_timestamp |{FAILED, [{SELECT from_utc_timestamp('2016-08-31', 'Asia/Seoul');, Unsupported}]} | -|get |{PASSED, [{SELECT get(array(1, 2, 3), 0);, OK}]} | -|get_json_object |{FAILED, [{SELECT get_json_object('{"a":"b"}', '$.a');, Unsupported}]} | -|getbit |{FAILED, [{SELECT getbit(11, 0);, Unsupported}]} | -|greatest |{FAILED, [{SELECT greatest(10, 9, 2, 4, 3);, Unsupported}]} | -|grouping |{FAILED, [{SELECT name, grouping(name), sum(age) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) GROUP BY cube(name);, Failed on something else. Check query manually}]} | -|grouping_id |{FAILED, [{SELECT name, grouping_id(), sum(age), avg(height) FROM VALUES (2, 'Alice', 165), (5, 'Bob', 180) people(age, name, height) GROUP BY cube(name, height);, Failed on something else. Check query manually}]}| -|hash |{FAILED, [{SELECT hash('Spark', array(123), 2);, Unsupported}]} | -|hex |{FAILED, [{SELECT hex(17);, Unsupported}]} | -|histogram_numeric |{FAILED, [{SELECT histogram_numeric(col, 5) FROM VALUES (0), (1), (2), (10) AS tab(col);, Unsupported}]} | -|hour |{FAILED, [{SELECT hour('2009-07-30 12:58:59');, Failed on native side}]} | -|hypot |{FAILED, [{SELECT hypot(3, 4);, Unsupported}]} | -|if |{PASSED, [{SELECT if(1 < 2, 'a', 'b');, OK}]} | -|ifnull |{FAILED, [{SELECT ifnull(NULL, array('2'));, Unsupported}]} | -|ilike |{FAILED, [{SELECT ilike('Spark', '_Park');, Failed on native side}]} | -|in |{PASSED, [{SELECT 1 in(1, 2, 3);, OK}]} | -|initcap |{PASSED, [{SELECT initcap('sPark sql');, OK}]} | -|inline |{FAILED, [{SELECT inline(array(struct(1, 'a'), struct(2, 'b')));, Unsupported}]} | -|inline_outer |{FAILED, [{SELECT inline_outer(array(struct(1, 'a'), struct(2, 'b')));, Unsupported}]} | -|input_file_block_length |{FAILED, [{SELECT input_file_block_length();, Unsupported}]} | -|input_file_block_start |{FAILED, [{SELECT input_file_block_start();, Unsupported}]} | -|input_file_name |{FAILED, [{SELECT input_file_name();, Unsupported}]} | -|instr |{PASSED, [{SELECT instr('SparkSQL', 'SQL');, OK}]} | -|int |{SKIPPED, [{, No examples found in spark.sessionState.functionRegistry}]} | -|isnan |{FAILED, [{SELECT isnan(cast('NaN' as double));, Unsupported}]} | -|isnotnull |{PASSED, [{SELECT isnotnull(1);, OK}]} | -|isnull |{PASSED, [{SELECT isnull(1);, OK}]} | -|java_method |{FAILED, [{SELECT java_method('java.util.UUID', 'randomUUID');, Unsupported}]} | -|json_array_length |{FAILED, [{SELECT json_array_length('[1,2,3,4]');, Unsupported}]} | -|json_object_keys |{FAILED, [{SELECT json_object_keys('{}');, Unsupported}]} | -|json_tuple |{FAILED, [{SELECT json_tuple('{"a":1, "b":2}', 'a', 'b');, Unsupported}]} | -|kurtosis |{FAILED, [{SELECT kurtosis(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col);, Unsupported}]} | -|lag |{FAILED, [{SELECT a, b, lag(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b);, Unsupported}]} | -|last |{FAILED, [{SELECT last(col) FROM VALUES (10), (5), (20) AS tab(col);, Unsupported}]} | -|last_day |{FAILED, [{SELECT last_day('2009-01-12');, Unsupported}]} | -|last_value |{FAILED, [{SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col);, Unsupported}]} | -|lcase |{PASSED, [{SELECT lcase('SparkSql');, OK}]} | -|lead |{FAILED, [{SELECT a, b, lead(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b);, Unsupported}]} | -|least |{FAILED, [{SELECT least(10, 9, 2, 4, 3);, Unsupported}]} | -|left |{FAILED, [{SELECT left('Spark SQL', 3);, Failed on native side}]} | -|len |{PASSED, [{SELECT len('Spark SQL ');, OK}]} | -|length |{PASSED, [{SELECT length('Spark SQL ');, OK}]} | -|levenshtein |{FAILED, [{SELECT levenshtein('kitten', 'sitting');, Unsupported}]} | -|like |{FAILED, [{SELECT like('Spark', '_park');, Failed on native side}]} | -|ln |{PASSED, [{SELECT ln(1);, OK}]} | -|localtimestamp |{FAILED, [{SELECT localtimestamp();, Failed on native side}]} | -|locate |{FAILED, [{SELECT locate('bar', 'foobarbar');, Unsupported}]} | -|log |{FAILED, [{SELECT log(10, 100);, Unsupported}]} | -|log10 |{PASSED, [{SELECT log10(10);, OK}]} | -|log1p |{FAILED, [{SELECT log1p(0);, Unsupported}]} | -|log2 |{PASSED, [{SELECT log2(2);, OK}]} | -|lower |{PASSED, [{SELECT lower('SparkSql');, OK}]} | -|lpad |{FAILED, [{SELECT lpad('hi', 5, '??');, Unsupported}]} | -|ltrim |{PASSED, [{SELECT ltrim(' SparkSQL ');, OK}]} | -|make_date |{FAILED, [{SELECT make_date(2013, 7, 15);, Unsupported}]} | -|make_dt_interval |{FAILED, [{SELECT make_dt_interval(1, 12, 30, 01.001001);, Unsupported}]} | -|make_interval |{FAILED, [{SELECT make_interval(100, 11, 1, 1, 12, 30, 01.001001);, Unsupported}]} | -|make_timestamp |{FAILED, [{SELECT make_timestamp(2014, 12, 28, 6, 30, 45.887);, Unsupported}]} | -|make_timestamp_ltz |{FAILED, [{SELECT make_timestamp_ltz(2014, 12, 28, 6, 30, 45.887);, Unsupported}]} | -|make_timestamp_ntz |{FAILED, [{SELECT make_timestamp_ntz(2014, 12, 28, 6, 30, 45.887);, Unsupported}]} | -|make_ym_interval |{FAILED, [{SELECT make_ym_interval(1, 2);, Unsupported}]} | -|map |{FAILED, [{SELECT map(1.0, '2', 3.0, '4');, Unsupported}]} | -|map_concat |{FAILED, [{SELECT map_concat(map(1, 'a', 2, 'b'), map(3, 'c'));, Unsupported}]} | -|map_contains_key |{FAILED, [{SELECT map_contains_key(map(1, 'a', 2, 'b'), 1);, Unsupported}]} | -|map_entries |{FAILED, [{SELECT map_entries(map(1, 'a', 2, 'b'));, Unsupported}]} | -|map_filter |{FAILED, [{SELECT map_filter(map(1, 0, 2, 2, 3, -1), (k, v) -> k > v);, Unsupported}]} | -|map_from_arrays |{FAILED, [{SELECT map_from_arrays(array(1.0, 3.0), array('2', '4'));, Unsupported}]} | -|map_from_entries |{FAILED, [{SELECT map_from_entries(array(struct(1, 'a'), struct(2, 'b')));, Unsupported}]} | -|map_keys |{FAILED, [{SELECT map_keys(map(1, 'a', 2, 'b'));, Unsupported}]} | -|map_values |{FAILED, [{SELECT map_values(map(1, 'a', 2, 'b'));, Unsupported}]} | -|map_zip_with |{FAILED, [{SELECT map_zip_with(map(1, 'a', 2, 'b'), map(1, 'x', 2, 'y'), (k, v1, v2) -> concat(v1, v2));, Unsupported}]} | -|mask |{FAILED, [{SELECT mask('abcd-EFGH-8765-4321');, Unsupported}]} | -|max |{FAILED, [{SELECT max(col) FROM VALUES (10), (50), (20) AS tab(col);, Unsupported}]} | -|max_by |{FAILED, [{SELECT max_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y);, Unsupported}]} | -|md5 |{PASSED, [{SELECT md5('Spark');, OK}]} | -|mean |{FAILED, [{SELECT mean(col) FROM VALUES (1), (2), (3) AS tab(col);, Unsupported}]} | -|median |{FAILED, [{SELECT median(col) FROM VALUES (0), (10) AS tab(col);, Unsupported}]} | -|min |{FAILED, [{SELECT min(col) FROM VALUES (10), (-1), (20) AS tab(col);, Unsupported}]} | -|min_by |{FAILED, [{SELECT min_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y);, Unsupported}]} | -|minute |{FAILED, [{SELECT minute('2009-07-30 12:58:59');, Failed on native side}]} | -|mod |{PASSED, [{SELECT 2 % 1.8;, OK}]} | -|mode |{FAILED, [{SELECT mode(col) FROM VALUES (0), (10), (10) AS tab(col);, Unsupported}]} | -|monotonically_increasing_id|{FAILED, [{SELECT monotonically_increasing_id();, Unsupported}]} | -|month |{FAILED, [{SELECT month('2016-07-30');, Unsupported}]} | -|months_between |{FAILED, [{SELECT months_between('1997-02-28 10:30:00', '1996-10-30');, Unsupported}]} | -|named_struct |{FAILED, [{SELECT named_struct("a", 1, "b", 2, "c", 3);, Unsupported}]} | -|nanvl |{FAILED, [{SELECT nanvl(cast('NaN' as double), 123);, Unsupported}]} | -|negative |{PASSED, [{SELECT negative(1);, OK}]} | -|next_day |{FAILED, [{SELECT next_day('2015-01-14', 'TU');, Unsupported}]} | -|not |{PASSED, [{SELECT not true;, OK}]} | -|now |{FAILED, [{SELECT now();, Failed on something else. Check query manually}]} | -|nth_value |{FAILED, [{SELECT a, b, nth_value(b, 2) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b);, Unsupported}]} | -|ntile |{FAILED, [{SELECT a, b, ntile(2) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b);, Unsupported}]} | -|nullif |{PASSED, [{SELECT nullif(2, 2);, OK}]} | -|nvl |{FAILED, [{SELECT nvl(NULL, array('2'));, Unsupported}]} | -|nvl2 |{PASSED, [{SELECT nvl2(NULL, 2, 1);, OK}]} | -|octet_length |{PASSED, [{SELECT octet_length('Spark SQL');, OK}]} | -|or |{PASSED, [{SELECT true or false;, OK}]} | -|overlay |{FAILED, [{SELECT overlay('Spark SQL' PLACING '_' FROM 6);, Unsupported}]} | -|parse_url |{FAILED, [{SELECT parse_url('http://spark.apache.org/path?query=1', 'HOST');, Unsupported}]} | -|percent_rank |{FAILED, [{SELECT a, b, percent_rank(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b);, Unsupported}]} | -|percentile |{FAILED, [{SELECT percentile(col, 0.3) FROM VALUES (0), (10) AS tab(col);, Unsupported}]} | -|percentile_approx |{FAILED, [{SELECT percentile_approx(col, array(0.5, 0.4, 0.1), 100) FROM VALUES (0), (1), (2), (10) AS tab(col);, Unsupported}]} | -|pi |{FAILED, [{SELECT pi();, Unsupported}]} | -|pmod |{FAILED, [{SELECT pmod(10, 3);, Unsupported}]} | -|posexplode |{FAILED, [{SELECT posexplode(array(10,20));, Unsupported}]} | -|posexplode_outer |{FAILED, [{SELECT posexplode_outer(array(10,20));, Unsupported}]} | -|position |{FAILED, [{SELECT position('bar', 'foobarbar');, Unsupported}]} | -|positive |{PASSED, [{SELECT positive(1);, OK}]} | -|pow |{PASSED, [{SELECT pow(2, 3);, OK}]} | -|power |{PASSED, [{SELECT power(2, 3);, OK}]} | -|printf |{FAILED, [{SELECT printf("Hello World %d %s", 100, "days");, Unsupported}]} | -|quarter |{FAILED, [{SELECT quarter('2016-08-31');, Unsupported}]} | -|radians |{FAILED, [{SELECT radians(180);, Unsupported}]} | -|raise_error |{FAILED, [{SELECT raise_error('custom error message');, Unsupported}]} | -|rand |{FAILED, [{SELECT rand();, Unsupported}]} | -|randn |{FAILED, [{SELECT randn();, Unsupported}]} | -|random |{FAILED, [{SELECT random();, Unsupported}]} | -|rank |{FAILED, [{SELECT a, b, rank(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b);, Unsupported}]} | -|reduce |{FAILED, [{SELECT reduce(array(1, 2, 3), 0, (acc, x) -> acc + x);, Unsupported}]} | -|reflect |{FAILED, [{SELECT reflect('java.util.UUID', 'randomUUID');, Unsupported}]} | -|regexp |{FAILED, [{SELECT regexp('%SystemDrive%\Users\John', '%SystemDrive%\\Users.*');, Unsupported}]} | -|regexp_count |{FAILED, [{SELECT regexp_count('Steven Jones and Stephen Smith are the best players', 'Ste(v|ph)en');, Unsupported}]} | -|regexp_extract |{FAILED, [{SELECT regexp_extract('100-200', '(\\d+)-(\\d+)', 1);, Unsupported}]} | -|regexp_extract_all |{FAILED, [{SELECT regexp_extract_all('100-200, 300-400', '(\\d+)-(\\d+)', 1);, Unsupported}]} | -|regexp_instr |{FAILED, [{SELECT regexp_instr('user@spark.apache.org', '@[^.]*');, Unsupported}]} | -|regexp_like |{FAILED, [{SELECT regexp_like('%SystemDrive%\Users\John', '%SystemDrive%\\Users.*');, Unsupported}]} | -|regexp_replace |{FAILED, [{SELECT regexp_replace('100-200', '(\\d+)', 'num');, Unsupported}]} | -|regexp_substr |{FAILED, [{SELECT regexp_substr('Steven Jones and Stephen Smith are the best players', 'Ste(v|ph)en');, Unsupported}]} | -|regr_avgx |{FAILED, [{SELECT regr_avgx(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x);, Unsupported}]} | -|regr_avgy |{FAILED, [{SELECT regr_avgy(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x);, Unsupported}]} | -|regr_count |{FAILED, [{SELECT regr_count(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x);, Unsupported}]} | -|regr_intercept |{FAILED, [{SELECT regr_intercept(y, x) FROM VALUES (1,1), (2,2), (3,3) AS tab(y, x);, Unsupported}]} | -|regr_r2 |{FAILED, [{SELECT regr_r2(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x);, Unsupported}]} | -|regr_slope |{FAILED, [{SELECT regr_slope(y, x) FROM VALUES (1,1), (2,2), (3,3) AS tab(y, x);, Unsupported}]} | -|regr_sxx |{FAILED, [{SELECT regr_sxx(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x);, Unsupported}]} | -|regr_sxy |{FAILED, [{SELECT regr_sxy(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x);, Unsupported}]} | -|regr_syy |{FAILED, [{SELECT regr_syy(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x);, Unsupported}]} | -|repeat |{PASSED, [{SELECT repeat('123', 2);, OK}]} | -|replace |{PASSED, [{SELECT replace('ABCabc', 'abc', 'DEF');, OK}]} | -|reverse |{PASSED, [{SELECT reverse('Spark SQL');, OK}]} | -|right |{FAILED, [{SELECT right('Spark SQL', 3);, Unsupported}]} | -|rint |{FAILED, [{SELECT rint(12.3456);, Unsupported}]} | -|rlike |{FAILED, [{SELECT rlike('%SystemDrive%\Users\John', '%SystemDrive%\\Users.*');, Unsupported}]} | -|round |{PASSED, [{SELECT round(2.5, 0);, OK}]} | -|row_number |{FAILED, [{SELECT a, b, row_number() OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b);, Unsupported}]} | -|rpad |{FAILED, [{SELECT rpad('hi', 5, '??');, Unsupported}]} | -|rtrim |{PASSED, [{SELECT rtrim(' SparkSQL ');, OK}]} | -|schema_of_csv |{FAILED, [{SELECT schema_of_csv('1,abc');, Unsupported}]} | -|schema_of_json |{FAILED, [{SELECT schema_of_json('[{"col":0}]');, Unsupported}]} | -|sec |{FAILED, [{SELECT sec(0);, Unsupported}]} | -|second |{FAILED, [{SELECT second('2009-07-30 12:58:59');, Failed on native side}]} | -|sentences |{FAILED, [{SELECT sentences('Hi there! Good morning.');, Unsupported}]} | -|sequence |{FAILED, [{SELECT sequence(1, 5);, Unsupported}]} | -|sha |{FAILED, [{SELECT sha('Spark');, Unsupported}]} | -|sha1 |{FAILED, [{SELECT sha1('Spark');, Unsupported}]} | -|sha2 |{FAILED, [{SELECT sha2('Spark', 256);, Unsupported}]} | -|shiftleft |{PASSED, [{SELECT shiftleft(2, 1);, OK}]} | -|shiftright |{PASSED, [{SELECT shiftright(4, 1);, OK}]} | -|shiftrightunsigned |{FAILED, [{SELECT shiftrightunsigned(4, 1);, Unsupported}]} | -|shuffle |{FAILED, [{SELECT shuffle(array(1, 20, 3, 5));, Unsupported}]} | -|sign |{PASSED, [{SELECT sign(40);, OK}]} | -|signum |{PASSED, [{SELECT signum(40);, OK}]} | -|sin |{PASSED, [{SELECT sin(0);, OK}]} | -|sinh |{FAILED, [{SELECT sinh(0);, Unsupported}]} | -|size |{FAILED, [{SELECT size(array('b', 'd', 'c', 'a'));, Unsupported}]} | -|skewness |{FAILED, [{SELECT skewness(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col);, Unsupported}]} | -|slice |{FAILED, [{SELECT slice(array(1, 2, 3, 4), 2, 2);, Unsupported}]} | -|smallint |{SKIPPED, [{, No examples found in spark.sessionState.functionRegistry}]} | -|some |{FAILED, [{SELECT some(col) FROM VALUES (true), (false), (false) AS tab(col);, Unsupported}]} | -|sort_array |{FAILED, [{SELECT sort_array(array('b', 'd', null, 'c', 'a'), true);, Unsupported}]} | -|soundex |{FAILED, [{SELECT soundex('Miller');, Unsupported}]} | -|space |{FAILED, [{SELECT concat(space(2), '1');, Unsupported}]} | -|spark_partition_id |{FAILED, [{SELECT spark_partition_id();, Unsupported}]} | -|split |{FAILED, [{SELECT split('oneAtwoBthreeC', '[ABC]');, Unsupported}]} | -|split_part |{FAILED, [{SELECT split_part('11.12.13', '.', 3);, Unsupported}]} | -|sqrt |{PASSED, [{SELECT sqrt(4);, OK}]} | -|stack |{FAILED, [{SELECT stack(2, 1, 2, 3);, Unsupported}]} | -|startswith |{FAILED, [{SELECT startswith('Spark SQL', 'Spark');, Failed on native side}]} | -|std |{FAILED, [{SELECT std(col) FROM VALUES (1), (2), (3) AS tab(col);, Unsupported}]} | -|stddev |{FAILED, [{SELECT stddev(col) FROM VALUES (1), (2), (3) AS tab(col);, Unsupported}]} | -|stddev_pop |{FAILED, [{SELECT stddev_pop(col) FROM VALUES (1), (2), (3) AS tab(col);, Unsupported}]} | -|stddev_samp |{FAILED, [{SELECT stddev_samp(col) FROM VALUES (1), (2), (3) AS tab(col);, Unsupported}]} | -|str_to_map |{FAILED, [{SELECT str_to_map('a:1,b:2,c:3', ',', ':');, Unsupported}]} | -|string |{SKIPPED, [{, No examples found in spark.sessionState.functionRegistry}]} | -|struct |{FAILED, [{SELECT struct(1, 2, 3);, Unsupported}]} | -|substr |{FAILED, [{SELECT substr('Spark SQL', 5);, Failed on native side}]} | -|substring |{FAILED, [{SELECT substring('Spark SQL', 5);, Failed on native side}]} | -|substring_index |{FAILED, [{SELECT substring_index('www.apache.org', '.', 2);, Unsupported}]} | -|sum |{FAILED, [{SELECT sum(col) FROM VALUES (5), (10), (15) AS tab(col);, Unsupported}]} | -|tan |{PASSED, [{SELECT tan(0);, OK}]} | -|tanh |{FAILED, [{SELECT tanh(0);, Unsupported}]} | -|timestamp |{SKIPPED, [{, No examples found in spark.sessionState.functionRegistry}]} | -|timestamp_micros |{FAILED, [{SELECT timestamp_micros(1230219000123123);, Unsupported}]} | -|timestamp_millis |{FAILED, [{SELECT timestamp_millis(1230219000123);, Unsupported}]} | -|timestamp_seconds |{FAILED, [{SELECT timestamp_seconds(1230219000);, Unsupported}]} | -|tinyint |{SKIPPED, [{, No examples found in spark.sessionState.functionRegistry}]} | -|to_binary |{FAILED, [{SELECT to_binary('abc', 'utf-8');, Unsupported}]} | -|to_char |{FAILED, [{SELECT to_char(454, '999');, Unsupported}]} | -|to_csv |{FAILED, [{SELECT to_csv(named_struct('a', 1, 'b', 2));, Unsupported}]} | -|to_date |{PASSED, [{SELECT to_date('2009-07-30 04:17:52');, OK}]} | -|to_json |{FAILED, [{SELECT to_json(named_struct('a', 1, 'b', 2));, Unsupported}]} | -|to_number |{FAILED, [{SELECT to_number('454', '999');, Unsupported}]} | -|to_timestamp |{PASSED, [{SELECT to_timestamp('2016-12-31 00:12:00');, OK}]} | -|to_timestamp_ltz |{PASSED, [{SELECT to_timestamp_ltz('2016-12-31 00:12:00');, OK}]} | -|to_timestamp_ntz |{FAILED, [{SELECT to_timestamp_ntz('2016-12-31 00:12:00');, Failed on native side}]} | -|to_unix_timestamp |{FAILED, [{SELECT to_unix_timestamp('2016-04-08', 'yyyy-MM-dd');, Unsupported}]} | -|to_utc_timestamp |{FAILED, [{SELECT to_utc_timestamp('2016-08-31', 'Asia/Seoul');, Unsupported}]} | -|transform |{FAILED, [{SELECT transform(array(1, 2, 3), x -> x + 1);, Unsupported}]} | -|transform_keys |{FAILED, [{SELECT transform_keys(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + 1);, Unsupported}]} | -|transform_values |{FAILED, [{SELECT transform_values(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> v + 1);, Unsupported}]} | -|translate |{PASSED, [{SELECT translate('AaBbCc', 'abc', '123');, OK}]} | -|trim |{PASSED, [{SELECT trim(' SparkSQL ');, OK}]} | -|trunc |{FAILED, [{SELECT trunc('2019-08-04', 'week');, Failed on native side}]} | -|try_add |{PASSED, [{SELECT try_add(1, 2);, OK}]} | -|try_avg |{FAILED, [{SELECT try_avg(col) FROM VALUES (1), (2), (3) AS tab(col);, Unsupported}]} | -|try_divide |{PASSED, [{SELECT try_divide(3, 2);, OK}]} | -|try_element_at |{FAILED, [{SELECT try_element_at(array(1, 2, 3), 2);, Unsupported}]} | -|try_multiply |{PASSED, [{SELECT try_multiply(2, 3);, OK}]} | -|try_subtract |{PASSED, [{SELECT try_subtract(2, 1);, OK}]} | -|try_sum |{FAILED, [{SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col);, Unsupported}]} | -|try_to_binary |{FAILED, [{SELECT try_to_binary('abc', 'utf-8');, Unsupported}]} | -|try_to_number |{FAILED, [{SELECT try_to_number('454', '999');, Unsupported}]} | -|try_to_timestamp |{PASSED, [{SELECT try_to_timestamp('2016-12-31 00:12:00');, OK}]} | -|typeof |{FAILED, [{SELECT typeof(1);, Unsupported}]} | -|ucase |{PASSED, [{SELECT ucase('SparkSql');, OK}]} | -|unbase64 |{FAILED, [{SELECT unbase64('U3BhcmsgU1FM');, Unsupported}]} | -|unhex |{FAILED, [{SELECT decode(unhex('537061726B2053514C'), 'UTF-8');, Unsupported}]} | -|unix_date |{FAILED, [{SELECT unix_date(DATE("1970-01-02"));, Unsupported}]} | -|unix_micros |{FAILED, [{SELECT unix_micros(TIMESTAMP('1970-01-01 00:00:01Z'));, Unsupported}]} | -|unix_millis |{FAILED, [{SELECT unix_millis(TIMESTAMP('1970-01-01 00:00:01Z'));, Unsupported}]} | -|unix_seconds |{FAILED, [{SELECT unix_seconds(TIMESTAMP('1970-01-01 00:00:01Z'));, Unsupported}]} | -|unix_timestamp |{FAILED, [{SELECT unix_timestamp();, Unsupported}]} | -|upper |{PASSED, [{SELECT upper('SparkSql');, OK}]} | -|url_decode |{FAILED, [{SELECT url_decode('https%3A%2F%2Fspark.apache.org');, Unsupported}]} | -|url_encode |{FAILED, [{SELECT url_encode('https://spark.apache.org');, Unsupported}]} | -|user |{PASSED, [{SELECT user();, OK}]} | -|uuid |{FAILED, [{SELECT uuid();, Unsupported}]} | -|var_pop |{FAILED, [{SELECT var_pop(col) FROM VALUES (1), (2), (3) AS tab(col);, Unsupported}]} | -|var_samp |{FAILED, [{SELECT var_samp(col) FROM VALUES (1), (2), (3) AS tab(col);, Unsupported}]} | -|variance |{FAILED, [{SELECT variance(col) FROM VALUES (1), (2), (3) AS tab(col);, Unsupported}]} | -|version |{FAILED, [{SELECT version();, Unsupported}]} | -|weekday |{FAILED, [{SELECT weekday('2009-07-30');, Unsupported}]} | -|weekofyear |{FAILED, [{SELECT weekofyear('2008-02-20');, Unsupported}]} | -|when |{PASSED, [{SELECT CASE WHEN 1 > 0 THEN 1 WHEN 2 > 0 THEN 2.0 ELSE 1.2 END;, OK}]} | -|width_bucket |{FAILED, [{SELECT width_bucket(5.3, 0.2, 10.6, 5);, Unsupported}]} | -|xpath |{FAILED, [{SELECT xpath('b1b2b3c1c2','a/b/text()');, Unsupported}]} | -|xpath_boolean |{FAILED, [{SELECT xpath_boolean('1','a/b');, Unsupported}]} | -|xpath_double |{FAILED, [{SELECT xpath_double('12', 'sum(a/b)');, Unsupported}]} | -|xpath_float |{FAILED, [{SELECT xpath_float('12', 'sum(a/b)');, Unsupported}]} | -|xpath_int |{FAILED, [{SELECT xpath_int('12', 'sum(a/b)');, Unsupported}]} | -|xpath_long |{FAILED, [{SELECT xpath_long('12', 'sum(a/b)');, Unsupported}]} | -|xpath_number |{FAILED, [{SELECT xpath_number('12', 'sum(a/b)');, Unsupported}]} | -|xpath_short |{FAILED, [{SELECT xpath_short('12', 'sum(a/b)');, Unsupported}]} | -|xpath_string |{FAILED, [{SELECT xpath_string('bcc','a/c');, Unsupported}]} | -|xxhash64 |{FAILED, [{SELECT xxhash64('Spark', array(123), 2);, Unsupported}]} | -|year |{PASSED, [{SELECT year('2016-07-30');, OK}]} | -|zip_with |{FAILED, [{SELECT zip_with(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x));, Unsupported}]} | -|| |{PASSED, [{SELECT 3 | 5;, OK}]} | -|~ |{FAILED, [{SELECT ~ 0;, Failed on native side}]} | -+---------------------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ diff --git a/doc/spark_builtin_expr_coverage_agg.txt b/doc/spark_builtin_expr_coverage_agg.txt deleted file mode 100644 index 013a84a66..000000000 --- a/doc/spark_builtin_expr_coverage_agg.txt +++ /dev/null @@ -1,9 +0,0 @@ -+-------+--------------------------------------------------------+---+ -|result |details |cnt| -+-------+--------------------------------------------------------+---+ -|FAILED |Unsupported |282| -|FAILED |Failed on native side |16 | -|FAILED |Failed on something else. Check query manually |4 | -|PASSED |OK |101| -|SKIPPED|No examples found in spark.sessionState.functionRegistry|12 | -+-------+--------------------------------------------------------+---+ diff --git a/docs/spark_builtin_expr_coverage.txt b/docs/spark_builtin_expr_coverage.txt new file mode 100644 index 000000000..7486e6380 --- /dev/null +++ b/docs/spark_builtin_expr_coverage.txt @@ -0,0 +1,419 @@ ++---------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------+-------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +|name |query |result |cometMessage |datafusionMessage | ++---------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------+-------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +|! |SELECT ! true; |PASSED |OK |std_err: SQL error: ParserError("Expected an expression:, found: !")\n | +|% |SELECT 2 % 1.8; |PASSED |OK |OK | +|& |SELECT 3 & 5; |PASSED |OK |OK | +|* |SELECT 2 * 3; |PASSED |OK |OK | +|+ |SELECT 1 + 2; |PASSED |OK |OK | +|- |SELECT 2 - 1; |PASSED |OK |OK | +|/ |SELECT 3 / 2; |PASSED |OK |OK | +|< |SELECT 1 < 2; |PASSED |OK |OK | +|<= |SELECT 2 <= 2; |PASSED |OK |OK | +|<=> |SELECT 2 <=> 2; |PASSED |OK |std_err: This feature is not implemented: Unsupported SQL binary operator Spaceship\n | +|= |SELECT 2 = 2; |PASSED |OK |OK | +|== |SELECT 2 == 2; |PASSED |OK |OK | +|> |SELECT 2 > 1; |PASSED |OK |OK | +|>= |SELECT 2 >= 1; |PASSED |OK |OK | +|^ |SELECT 3 ^ 5; |PASSED |OK |OK | +|abs |SELECT abs(-1); |PASSED |OK |OK | +|acos |SELECT acos(1); |PASSED |OK |OK | +|acosh |SELECT acosh(1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|add_months |SELECT add_months('2016-08-31', 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'add_months'.\nDid you mean 'acos'?\n | +|aes_decrypt |SELECT aes_decrypt(unhex('83F16B2AA704794132802D248E6BFD4E380078182D1544813898AC97E709B28A94'), '0000111122223333'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'aes_decrypt'.\nDid you mean 'list_except'?\n | +|aes_encrypt |SELECT hex(aes_encrypt('Spark', '0000111122223333')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'hex'.\nDid you mean 'exp'?\n | +|aggregate |SELECT aggregate(array(1, 2, 3), 0, (acc, x) -> acc + x); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|and |SELECT true and true; |PASSED |OK |OK | +|any |SELECT any(col) FROM VALUES (true), (false), (false) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'any'.\nDid you mean 'abs'?\n | +|any_value |SELECT any_value(col) FROM VALUES (10), (5), (20) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'any_value'.\nDid you mean 'LAST_VALUE'?\n | +|approx_count_distinct |SELECT approx_count_distinct(col1) FROM VALUES (1), (1), (2), (2), (3) tab(col1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'approx_count_distinct'.\nDid you mean 'APPROX_DISTINCT'?\n | +|approx_percentile |SELECT approx_percentile(col, array(0.5, 0.4, 0.1), 100) FROM VALUES (0), (1), (2), (10) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 0.5")\n | +|array |SELECT array(1, 2, 3); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|array_agg |SELECT array_agg(col) FROM VALUES (1), (2), (1) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|array_append |SELECT array_append(array('b', 'd', 'c', 'a'), 'd'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 'b'")\n | +|array_compact |SELECT array_compact(array(1, 2, 3, null)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|array_contains |SELECT array_contains(array(1, 2, 3), 2); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|array_distinct |SELECT array_distinct(array(1, 2, 3, null, 3)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|array_except |SELECT array_except(array(1, 2, 3), array(1, 3, 5)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|array_insert |SELECT array_insert(array(1, 2, 3, 4), 5, 5); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|array_intersect |SELECT array_intersect(array(1, 2, 3), array(1, 3, 5)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|array_join |SELECT array_join(array('hello', 'world'), ' '); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 'hello'")\n | +|array_max |SELECT array_max(array(1, 20, null, 3)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|array_min |SELECT array_min(array(1, 20, null, 3)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|array_position |SELECT array_position(array(3, 2, 1), 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 3")\n | +|array_remove |SELECT array_remove(array(1, 2, 3, null, 3), 3); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|array_repeat |SELECT array_repeat('123', 2); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|array_size |SELECT array_size(array('b', 'd', 'c', 'a')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 'b'")\n | +|array_sort |SELECT array_sort(array(5, 6, 1), (left, right) -> case when left < right then -1 when left > right then 1 else 0 end); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 5")\n | +|array_union |SELECT array_union(array(1, 2, 3), array(1, 3, 5)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|arrays_overlap |SELECT arrays_overlap(array(1, 2, 3), array(3, 4, 5)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|arrays_zip |SELECT arrays_zip(array(1, 2, 3), array(2, 3, 4)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|ascii |SELECT ascii('222'); |PASSED |OK |OK | +|asin |SELECT asin(0); |PASSED |OK |OK | +|asinh |SELECT asinh(0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|assert_true |SELECT assert_true(0 < 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'assert_true'.\nDid you mean 'date_trunc'?\n | +|atan |SELECT atan(0); |PASSED |OK |OK | +|atan2 |SELECT atan2(0, 0); |PASSED |OK |OK | +|atanh |SELECT atanh(0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|avg |SELECT avg(col) FROM VALUES (1), (2), (3) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|base64 |SELECT base64('Spark SQL'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'base64'.\nDid you mean 'asinh'?\n | +|bigint | |SKIPPED|No examples found in spark.sessionState.functionRegistry | | +|bin |SELECT bin(13); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'bin'.\nDid you mean 'sin'?\n | +|binary | |SKIPPED|No examples found in spark.sessionState.functionRegistry | | +|bit_and |SELECT bit_and(col) FROM VALUES (3), (5) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|bit_count |SELECT bit_count(0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'bit_count'.\nDid you mean 'COUNT'?\n | +|bit_get |SELECT bit_get(11, 0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'bit_get'.\nDid you mean 'BIT_AND'?\n | +|bit_length |SELECT bit_length('Spark SQL'); |PASSED |OK |OK | +|bit_or |SELECT bit_or(col) FROM VALUES (3), (5) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|bit_xor |SELECT bit_xor(col) FROM VALUES (3), (5) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|bool_and |SELECT bool_and(col) FROM VALUES (true), (true), (true) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|bool_or |SELECT bool_or(col) FROM VALUES (true), (false), (false) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|boolean | |SKIPPED|No examples found in spark.sessionState.functionRegistry | | +|bround |SELECT bround(2.5, 0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'bround'.\nDid you mean 'round'?\n | +|btrim |SELECT btrim(' SparkSQL '); |PASSED |OK |OK | +|cardinality |SELECT cardinality(array('b', 'd', 'c', 'a')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 'b'")\n | +|cast |SELECT cast('10' as int); |PASSED |OK |OK | +|cbrt |SELECT cbrt(27.0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|ceil |SELECT ceil(-0.1); |PASSED |OK |OK | +|ceiling |SELECT ceiling(-0.1); |PASSED |OK |std_err: Error during planning: Invalid function 'ceiling'.\nDid you mean 'ceil'?\n | +|char |SELECT char(65); |PASSED |OK |std_err: Error during planning: Invalid function 'char'.\nDid you mean 'chr'?\n | +|char_length |SELECT char_length('Spark SQL '); |PASSED |OK |OK | +|character_length |SELECT character_length('Spark SQL '); |PASSED |OK |OK | +|chr |SELECT chr(65); |PASSED |OK |OK | +|coalesce |SELECT coalesce(NULL, 1, NULL); |PASSED |OK |OK | +|collect_list |SELECT collect_list(col) FROM VALUES (1), (2), (1) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'collect_list'.\nDid you mean 'make_list'?\n | +|collect_set |SELECT collect_set(col) FROM VALUES (1), (2), (1) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'collect_set'.\nDid you mean 'coalesce'?\n | +|concat |SELECT concat('Spark', 'SQL'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|concat_ws |SELECT concat_ws(' ', 'Spark', 'SQL'); |PASSED |OK |OK | +|contains |SELECT contains('Spark SQL', 'Spark'); |FAILED |Failed on native side: found CometNativeException |std_err: Error during planning: Invalid function 'contains'.\nDid you mean 'concat_ws'?\n | +|conv |SELECT conv('100', 2, 10); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'conv'.\nDid you mean 'cot'?\n | +|convert_timezone |SELECT convert_timezone('Europe/Brussels', 'America/Los_Angeles', timestamp_ntz'2021-12-06 00:00:00'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected ), found: '2021-12-06 00:00:00'")\n | +|corr |SELECT corr(c1, c2) FROM VALUES (3, 2), (3, 3), (6, 4) as tab(c1, c2); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|cos |SELECT cos(0); |PASSED |OK |OK | +|cosh |SELECT cosh(0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|cot |SELECT cot(1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|count |SELECT count(*) FROM VALUES (NULL), (5), (5), (20) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|count_if |SELECT count_if(col % 2 = 0) FROM VALUES (NULL), (0), (1), (2), (3) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'count_if'.\nDid you mean 'COUNT'?\n | +|count_min_sketch |SELECT hex(count_min_sketch(col, 0.5d, 0.5d, 1)) FROM VALUES (1), (2), (1) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: SQL error: ParserError("Expected ), found: d")\n | +|covar_pop |SELECT covar_pop(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|covar_samp |SELECT covar_samp(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|crc32 |SELECT crc32('Spark'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'crc32'.\nDid you mean 'ascii'?\n | +|csc |SELECT csc(1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'csc'.\nDid you mean 'chr'?\n | +|cume_dist |SELECT a, b, cume_dist() OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|curdate |SELECT curdate(); |PASSED |OK |std_err: Error during planning: Invalid function 'curdate'.\nDid you mean 'to_date'?\n | +|current_catalog |SELECT current_catalog(); |PASSED |OK |std_err: SQL error: ParserError("Expected end of statement, found: (")\n | +|current_database |SELECT current_database(); |PASSED |OK |std_err: Error during planning: Invalid function 'current_database'.\nDid you mean 'current_date'?\n | +|current_date |SELECT current_date(); |PASSED |OK |OK | +|current_schema |SELECT current_schema(); |PASSED |OK |std_err: Error during planning: Invalid function 'current_schema'.\nDid you mean 'current_time'?\n | +|current_timestamp |SELECT current_timestamp(); |FAILED |\nResults do not match for query:\nTimezone: sun.util.calendar.ZoneInfo[id="America/Los_Angeles",offset=-28800000,dstSavings=3600000,useDaylight=true,transitions=185,lastRule=java.util.SimpleTimeZone[id=America/Los_Angeles,offset=-28800000,dstSavings=3600000,useDaylight=true,startYear=0,startMode=3,startMonth=2,startDay=8,startDayOfWeek=1,startTime=7200000,startTimeMode=0,endMode=3,endMonth=10,endDay=1,endDayOfWeek=1,endTime=7200000,endTimeMode=0]]\nTimezone Env: \n\n== Parsed Logical Plan ==\nProject [current_timestamp() AS current_timestamp()#3031, x#3015]\n+- SubqueryAlias tbl\n +- View (`tbl`, [x#3015])\n +- Relation [x#3015] parquet\n\n== Analyzed Logical Plan ==\ncurrent_timestamp(): timestamp, x: string\nProject [current_timestamp() AS current_timestamp()#3031, x#3015]\n+- SubqueryAlias tbl\n +- View (`tbl`, [x#3015])\n +- Relation [x#3015] parquet\n\n== Optimized Logical Plan ==\nProject [2024-05-10 10:13:21.77322 AS current_timestamp()#3031, x#3015]\n+- Relation [x#3015] parquet\n\n== Physical Plan ==\n*(1) ColumnarToRow\n+- CometProject [current_timestamp()#3031, x#3015], [2024-05-10 10:13:21.77322 AS current_timestamp()#3031, x#3015]\n +- CometScan parquet [x#3015] Batched: true, DataFilters: [], Format: CometParquet, Location: InMemoryFileIndex(1 paths)[file:/private/var/folders/6f/_s1vnnd55zgfkx7zlwnrnv0h0000gn/T/spark-62..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n\n== Results ==\n\n== Results ==\n!== Correct Answer - 1 == == Spark Answer - 1 ==\n struct struct\n![2024-05-10 10:13:21.749547,dummy] [2024-05-10 10:13:21.77322,dummy]\n \n |std_err: Error during planning: Invalid function 'current_timestamp'.\nDid you mean 'current_time'?\n | +|current_timezone |SELECT current_timezone(); |PASSED |OK |std_err: Error during planning: Invalid function 'current_timezone'.\nDid you mean 'current_time'?\n | +|current_user |SELECT current_user(); |PASSED |OK |std_err: SQL error: ParserError("Expected end of statement, found: (")\n | +|date | |SKIPPED|No examples found in spark.sessionState.functionRegistry | | +|date_add |SELECT date_add('2016-07-30', 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'date_add'.\nDid you mean 'date_bin'?\n | +|date_diff |SELECT date_diff('2009-07-31', '2009-07-30'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'date_diff'.\nDid you mean 'date_bin'?\n | +|date_format |SELECT date_format('2016-04-08', 'y'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|date_from_unix_date |SELECT date_from_unix_date(1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'date_from_unix_date'.\nDid you mean 'from_unixtime'?\n | +|date_part |SELECT date_part('YEAR', TIMESTAMP '2019-08-12 01:00:00.123456'); |PASSED |OK |OK | +|date_sub |SELECT date_sub('2016-07-30', 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'date_sub'.\nDid you mean 'date_bin'?\n | +|date_trunc |SELECT date_trunc('YEAR', '2015-03-05T09:32:05.359'); |FAILED |Failed on native side: found CometNativeException | | +|dateadd |SELECT dateadd('2016-07-30', 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'dateadd'.\nDid you mean 'datepart'?\n | +|datediff |SELECT datediff('2009-07-31', '2009-07-30'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'datediff'.\nDid you mean 'datepart'?\n | +|datepart |SELECT datepart('YEAR', TIMESTAMP '2019-08-12 01:00:00.123456'); |PASSED |OK |OK | +|day |SELECT day('2009-07-30'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'day'.\nDid you mean 'today'?\n | +|dayofmonth |SELECT dayofmonth('2009-07-30'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'dayofmonth'.\nDid you mean 'datepart'?\n | +|dayofweek |SELECT dayofweek('2009-07-30'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'dayofweek'.\nDid you mean 'degrees'?\n | +|dayofyear |SELECT dayofyear('2016-04-09'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'dayofyear'.\nDid you mean 'to_char'?\n | +|decimal | |SKIPPED|No examples found in spark.sessionState.functionRegistry | | +|decode |SELECT decode(encode('abc', 'utf-8'), 'utf-8'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: There is no built-in encoding named 'utf-8', currently supported encodings are: base64, hex\n | +|degrees |SELECT degrees(3.141592653589793); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|dense_rank |SELECT a, b, dense_rank(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: No function matches the given name and argument types 'DENSE_RANK(Int32)'. You might need to add explicit type casts.\n\tCandidate functions:\n\tDENSE_RANK()\n | +|div |SELECT 3 div 2; |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("No infix parser for token Word(Word { value: \"div\", quote_style: None, keyword: DIV })")\n | +|double | |SKIPPED|No examples found in spark.sessionState.functionRegistry | | +|e |SELECT e(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'e'.\nDid you mean 'exp'?\n | +|element_at |SELECT element_at(array(1, 2, 3), 2); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|elt |SELECT elt(1, 'scala', 'java'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'elt'.\nDid you mean 'ln'?\n | +|encode |SELECT encode('abc', 'utf-8'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: There is no built-in encoding named 'utf-8', currently supported encodings are: base64, hex\n | +|endswith |SELECT endswith('Spark SQL', 'SQL'); |FAILED |Failed on native side: found CometNativeException |std_err: Error during planning: Invalid function 'endswith'.\nDid you mean 'ends_with'?\n | +|equal_null |SELECT equal_null(3, 3); |PASSED |OK |std_err: Error during planning: Invalid function 'equal_null'.\nDid you mean 'ifnull'?\n | +|every |SELECT every(col) FROM VALUES (true), (true), (true) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'every'.\nDid you mean 'reverse'?\n | +|exists |SELECT exists(array(1, 2, 3), x -> x % 2 == 0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: array")\n | +|exp |SELECT exp(0); |PASSED |OK |OK | +|explode |SELECT explode(array(10, 20)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 10")\n | +|explode_outer |SELECT explode_outer(array(10, 20)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 10")\n | +|expm1 |SELECT expm1(0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'expm1'.\nDid you mean 'exp'?\n | +|extract |SELECT extract(YEAR FROM TIMESTAMP '2019-08-12 01:00:00.123456'); |PASSED |OK |OK | +|factorial |SELECT factorial(5); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|filter |SELECT filter(array(1, 2, 3), x -> x % 2 == 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|find_in_set |SELECT find_in_set('ab','abc,b,ab,c,def'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|first |SELECT first(col) FROM VALUES (10), (5), (20) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'first'.\nDid you mean 'right'?\n | +|first_value |SELECT first_value(col) FROM VALUES (10), (5), (20) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|flatten |SELECT flatten(array(array(1, 2), array(3, 4))); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: array")\n | +|float | |SKIPPED|No examples found in spark.sessionState.functionRegistry | | +|floor |SELECT floor(-0.1); |PASSED |OK |OK | +|forall |SELECT forall(array(1, 2, 3), x -> x % 2 == 0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|format_number |SELECT format_number(12332.123456, 4); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'format_number'.\nDid you mean 'FIRST_VALUE'?\n | +|format_string |SELECT format_string("Hello World %d %s", 100, "days"); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'format_string'.\nDid you mean 'array_to_string'?\n | +|from_csv |SELECT from_csv('1, 0.8', 'a INT, b DOUBLE'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'from_csv'.\nDid you mean 'arrow_cast'?\n | +|from_json |SELECT from_json('{"a":1, "b":0.8}', 'a INT, b DOUBLE'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'from_json'.\nDid you mean 'floor'?\n | +|from_unixtime |SELECT from_unixtime(0, 'yyyy-MM-dd HH:mm:ss'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: No function matches the given name and argument types 'from_unixtime(Int64, Utf8)'. You might need to add explicit type casts.\n\tCandidate functions:\n\tfrom_unixtime(Int64)\n | +|from_utc_timestamp |SELECT from_utc_timestamp('2016-08-31', 'Asia/Seoul'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'from_utc_timestamp'.\nDid you mean 'to_timestamp'?\n | +|get |SELECT get(array(1, 2, 3), 0); |PASSED |OK |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|get_json_object |SELECT get_json_object('{"a":"b"}', '$.a'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'get_json_object'.\nDid you mean 'list_pop_back'?\n | +|getbit |SELECT getbit(11, 0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'getbit'.\nDid you mean 'ceil'?\n | +|greatest |SELECT greatest(10, 9, 2, 4, 3); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'greatest'.\nDid you mean 'repeat'?\n | +|grouping |SELECT name, grouping(name), sum(age) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) GROUP BY cube(name); |FAILED |[MISSING_AGGREGATION] The non-aggregating expression "age" is based on columns which are not participating in the GROUP BY clause.\nAdd the columns or the expression to the GROUP BY, aggregate the expression, or use "any_value(age)" if you do not care which of the values within a group is returned.;\nAggregate [name#2277, spark_grouping_id#2276L], [age#2273, name#2277]\n+- Expand [[age#2273, name#2274, name#2275, 0], [age#2273, name#2274, null, 1]], [age#2273, name#2274, name#2277, spark_grouping_id#2276L]\n +- Project [age#2273, name#2274, name#2274 AS name#2275]\n +- SubqueryAlias people\n +- LocalRelation [age#2273, name#2274]\n | | +|grouping_id |SELECT name, grouping_id(), sum(age), avg(height) FROM VALUES (2, 'Alice', 165), (5, 'Bob', 180) people(age, name, height) GROUP BY cube(name, height);|FAILED |[MISSING_AGGREGATION] The non-aggregating expression "age" is based on columns which are not participating in the GROUP BY clause.\nAdd the columns or the expression to the GROUP BY, aggregate the expression, or use "any_value(age)" if you do not care which of the values within a group is returned.;\nAggregate [name#7432, height#7433, spark_grouping_id#7431L], [age#7426, name#7432, height#7433]\n+- Expand [[age#7426, name#7427, height#7428, name#7429, height#7430, 0], [age#7426, name#7427, height#7428, name#7429, null, 1], [age#7426, name#7427, height#7428, null, height#7430, 2], [age#7426, name#7427, height#7428, null, null, 3]], [age#7426, name#7427, height#7428, name#7432, height#7433, spark_grouping_id#7431L]\n +- Project [age#7426, name#7427, height#7428, name#7427 AS name#7429, height#7428 AS height#7430]\n +- SubqueryAlias people\n +- LocalRelation [age#7426, name#7427, height#7428]\n | | +|hash |SELECT hash('Spark', array(123), 2); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 123")\n | +|hex |SELECT hex(17); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'hex'.\nDid you mean 'exp'?\n | +|histogram_numeric |SELECT histogram_numeric(col, 5) FROM VALUES (0), (1), (2), (10) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'histogram_numeric'.\nDid you mean 'list_remove_n'?\n | +|hour |SELECT hour('2009-07-30 12:58:59'); |FAILED |Failed on native side: found CometNativeException |std_err: Error during planning: Invalid function 'hour'.\nDid you mean 'CORR'?\n | +|hypot |SELECT hypot(3, 4); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'hypot'.\nDid you mean 'pow'?\n | +|if |SELECT if(1 < 2, 'a', 'b'); |PASSED |OK |std_err: Error during planning: Invalid function 'if'.\nDid you mean 'sin'?\n | +|ifnull |SELECT ifnull(NULL, array('2')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: '2'")\n | +|ilike |SELECT ilike('Spark', '_Park'); |FAILED |Failed on native side: found CometNativeException |std_err: Error during planning: Invalid function 'ilike'.\nDid you mean 'lpad'?\n | +|in |SELECT 1 in(1, 2, 3); |PASSED |OK |OK | +|initcap |SELECT initcap('sPark sql'); |PASSED |OK |OK | +|inline |SELECT inline(array(struct(1, 'a'), struct(2, 'b'))); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: struct")\n | +|inline_outer |SELECT inline_outer(array(struct(1, 'a'), struct(2, 'b'))); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: struct")\n | +|input_file_block_length |SELECT input_file_block_length(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'input_file_block_length'.\nDid you mean 'octet_length'?\n | +|input_file_block_start |SELECT input_file_block_start(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'input_file_block_start'.\nDid you mean 'list_replace_all'?\n | +|input_file_name |SELECT input_file_name(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'input_file_name'.\nDid you mean 'bit_length'?\n | +|instr |SELECT instr('SparkSQL', 'SQL'); |PASSED |OK |OK | +|int | |SKIPPED|No examples found in spark.sessionState.functionRegistry | | +|isnan |SELECT isnan(cast('NaN' as double)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|isnotnull |SELECT isnotnull(1); |PASSED |OK |std_err: Error during planning: Invalid function 'isnotnull'.\nDid you mean 'ifnull'?\n | +|isnull |SELECT isnull(1); |PASSED |OK |std_err: Error during planning: Invalid function 'isnull'.\nDid you mean 'ifnull'?\n | +|java_method |SELECT java_method('java.util.UUID', 'randomUUID'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'java_method'.\nDid you mean 'make_date'?\n | +|json_array_length |SELECT json_array_length('[1,2,3,4]'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'json_array_length'.\nDid you mean 'array_length'?\n | +|json_object_keys |SELECT json_object_keys('{}'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'json_object_keys'.\nDid you mean 'concat_ws'?\n | +|json_tuple |SELECT json_tuple('{"a":1, "b":2}', 'a', 'b'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'json_tuple'.\nDid you mean 'strpos'?\n | +|kurtosis |SELECT kurtosis(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'kurtosis'.\nDid you mean 'rtrim'?\n | +|lag |SELECT a, b, lag(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|last |SELECT last(col) FROM VALUES (10), (5), (20) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'last'.\nDid you mean 'left'?\n | +|last_day |SELECT last_day('2009-01-12'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'last_day'.\nDid you mean 'list_cat'?\n | +|last_value |SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|lcase |SELECT lcase('SparkSql'); |PASSED |OK |std_err: Error during planning: Invalid function 'lcase'.\nDid you mean 'acos'?\n | +|lead |SELECT a, b, lead(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|least |SELECT least(10, 9, 2, 4, 3); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'least'.\nDid you mean 'left'?\n | +|left |SELECT left('Spark SQL', 3); |FAILED |Failed on native side: found CometNativeException | | +|len |SELECT len('Spark SQL '); |PASSED |OK |std_err: Error during planning: Invalid function 'len'.\nDid you mean 'ln'?\n | +|length |SELECT length('Spark SQL '); |PASSED |OK |OK | +|levenshtein |SELECT levenshtein('kitten', 'sitting'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|like |SELECT like('Spark', '_park'); |FAILED |Failed on native side: found CometNativeException |std_err: Error during planning: Invalid function 'like'.\nDid you mean 'lower'?\n | +|ln |SELECT ln(1); |PASSED |OK |OK | +|localtimestamp |SELECT localtimestamp(); |FAILED |Failed on native side: found CometNativeException |std_err: Error during planning: Invalid function 'localtimestamp'.\nDid you mean 'to_timestamp'?\n | +|locate |SELECT locate('bar', 'foobarbar'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'locate'.\nDid you mean 'to_date'?\n | +|log |SELECT log(10, 100); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|log10 |SELECT log10(10); |PASSED |OK |OK | +|log1p |SELECT log1p(0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'log1p'.\nDid you mean 'log10'?\n | +|log2 |SELECT log2(2); |PASSED |OK |OK | +|lower |SELECT lower('SparkSql'); |PASSED |OK |OK | +|lpad |SELECT lpad('hi', 5, '??'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|ltrim |SELECT ltrim(' SparkSQL '); |PASSED |OK |OK | +|make_date |SELECT make_date(2013, 7, 15); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|make_dt_interval |SELECT make_dt_interval(1, 12, 30, 01.001001); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'make_dt_interval'.\nDid you mean 'make_date'?\n | +|make_interval |SELECT make_interval(100, 11, 1, 1, 12, 30, 01.001001); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'make_interval'.\nDid you mean 'make_array'?\n | +|make_timestamp |SELECT make_timestamp(2014, 12, 28, 6, 30, 45.887); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'make_timestamp'.\nDid you mean 'to_timestamp'?\n | +|make_timestamp_ltz |SELECT make_timestamp_ltz(2014, 12, 28, 6, 30, 45.887); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'make_timestamp_ltz'.\nDid you mean 'to_timestamp'?\n | +|make_timestamp_ntz |SELECT make_timestamp_ntz(2014, 12, 28, 6, 30, 45.887); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'make_timestamp_ntz'.\nDid you mean 'to_timestamp'?\n | +|make_ym_interval |SELECT make_ym_interval(1, 2); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'make_ym_interval'.\nDid you mean 'array_intersect'?\n | +|map |SELECT map(1.0, '2', 3.0, '4'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'map'.\nDid you mean 'MAX'?\n | +|map_concat |SELECT map_concat(map(1, 'a', 2, 'b'), map(3, 'c')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'map_concat'.\nDid you mean 'array_concat'?\n | +|map_contains_key |SELECT map_contains_key(map(1, 'a', 2, 'b'), 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'map_contains_key'.\nDid you mean 'array_contains'?\n | +|map_entries |SELECT map_entries(map(1, 'a', 2, 'b')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'map_entries'.\nDid you mean 'make_array'?\n | +|map_filter |SELECT map_filter(map(1, 0, 2, 2, 3, -1), (k, v) -> k > v); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'map_filter'.\nDid you mean 'make_date'?\n | +|map_from_arrays |SELECT map_from_arrays(array(1.0, 3.0), array('2', '4')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1.0")\n | +|map_from_entries |SELECT map_from_entries(array(struct(1, 'a'), struct(2, 'b'))); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: struct")\n | +|map_keys |SELECT map_keys(map(1, 'a', 2, 'b')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'map_keys'.\nDid you mean 'make_list'?\n | +|map_values |SELECT map_values(map(1, 'a', 2, 'b')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'map_values'.\nDid you mean 'LAST_VALUE'?\n | +|map_zip_with |SELECT map_zip_with(map(1, 'a', 2, 'b'), map(1, 'x', 2, 'y'), (k, v1, v2) -> concat(v1, v2)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'map_zip_with'.\nDid you mean 'starts_with'?\n | +|mask |SELECT mask('abcd-EFGH-8765-4321'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'mask'.\nDid you mean 'MAX'?\n | +|max |SELECT max(col) FROM VALUES (10), (50), (20) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|max_by |SELECT max_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'max_by'.\nDid you mean 'MAX'?\n | +|md5 |SELECT md5('Spark'); |PASSED |OK |OK | +|mean |SELECT mean(col) FROM VALUES (1), (2), (3) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|median |SELECT median(col) FROM VALUES (0), (10) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|min |SELECT min(col) FROM VALUES (10), (-1), (20) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|min_by |SELECT min_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'min_by'.\nDid you mean 'MIN'?\n | +|minute |SELECT minute('2009-07-30 12:58:59'); |FAILED |Failed on native side: found CometNativeException |std_err: Error during planning: Invalid function 'minute'.\nDid you mean 'instr'?\n | +|mod |SELECT 2 % 1.8; |PASSED |OK |OK | +|mode |SELECT mode(col) FROM VALUES (0), (10), (10) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'mode'.\nDid you mean 'md5'?\n | +|monotonically_increasing_id|SELECT monotonically_increasing_id(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'monotonically_increasing_id'.\nDid you mean 'array_intersect'?\n | +|month |SELECT month('2016-07-30'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'month'.\nDid you mean 'tanh'?\n | +|months_between |SELECT months_between('1997-02-28 10:30:00', '1996-10-30'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'months_between'.\nDid you mean 'NTH_VALUE'?\n | +|named_struct |SELECT named_struct("a", 1, "b", 2, "c", 3); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Schema error: No field named a.\n | +|nanvl |SELECT nanvl(cast('NaN' as double), 123); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|negative |SELECT negative(1); |PASSED |OK |std_err: Error during planning: Invalid function 'negative'.\nDid you mean 'nanvl'?\n | +|next_day |SELECT next_day('2015-01-14', 'TU'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'next_day'.\nDid you mean 'today'?\n | +|not |SELECT not true; |PASSED |OK |OK | +|now |SELECT now(); |FAILED |\nResults do not match for query:\nTimezone: sun.util.calendar.ZoneInfo[id="America/Los_Angeles",offset=-28800000,dstSavings=3600000,useDaylight=true,transitions=185,lastRule=java.util.SimpleTimeZone[id=America/Los_Angeles,offset=-28800000,dstSavings=3600000,useDaylight=true,startYear=0,startMode=3,startMonth=2,startDay=8,startDayOfWeek=1,startTime=7200000,startTimeMode=0,endMode=3,endMonth=10,endDay=1,endDayOfWeek=1,endTime=7200000,endTimeMode=0]]\nTimezone Env: \n\n== Parsed Logical Plan ==\nProject [now() AS now()#4526, x#4510]\n+- SubqueryAlias tbl\n +- View (`tbl`, [x#4510])\n +- Relation [x#4510] parquet\n\n== Analyzed Logical Plan ==\nnow(): timestamp, x: string\nProject [now() AS now()#4526, x#4510]\n+- SubqueryAlias tbl\n +- View (`tbl`, [x#4510])\n +- Relation [x#4510] parquet\n\n== Optimized Logical Plan ==\nProject [2024-05-10 10:13:29.302484 AS now()#4526, x#4510]\n+- Relation [x#4510] parquet\n\n== Physical Plan ==\n*(1) ColumnarToRow\n+- CometProject [now()#4526, x#4510], [2024-05-10 10:13:29.302484 AS now()#4526, x#4510]\n +- CometScan parquet [x#4510] Batched: true, DataFilters: [], Format: CometParquet, Location: InMemoryFileIndex(1 paths)[file:/private/var/folders/6f/_s1vnnd55zgfkx7zlwnrnv0h0000gn/T/spark-28..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n\n== Results ==\n\n== Results ==\n!== Correct Answer - 1 == == Spark Answer - 1 ==\n struct struct\n![2024-05-10 10:13:29.285019,dummy] [2024-05-10 10:13:29.302484,dummy]\n \n | | +|nth_value |SELECT a, b, nth_value(b, 2) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|ntile |SELECT a, b, ntile(2) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|nullif |SELECT nullif(2, 2); |PASSED |OK |OK | +|nvl |SELECT nvl(NULL, array('2')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: '2'")\n | +|nvl2 |SELECT nvl2(NULL, 2, 1); |PASSED |OK |OK | +|octet_length |SELECT octet_length('Spark SQL'); |PASSED |OK |OK | +|or |SELECT true or false; |PASSED |OK |OK | +|overlay |SELECT overlay('Spark SQL' PLACING '_' FROM 6); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|parse_url |SELECT parse_url('http://spark.apache.org/path?query=1', 'HOST'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'parse_url'.\nDid you mean 'date_part'?\n | +|percent_rank |SELECT a, b, percent_rank(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: No function matches the given name and argument types 'PERCENT_RANK(Int32)'. You might need to add explicit type casts.\n\tCandidate functions:\n\tPERCENT_RANK()\n | +|percentile |SELECT percentile(col, 0.3) FROM VALUES (0), (10) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'percentile'.\nDid you mean 'current_time'?\n | +|percentile_approx |SELECT percentile_approx(col, array(0.5, 0.4, 0.1), 100) FROM VALUES (0), (1), (2), (10) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 0.5")\n | +|pi |SELECT pi(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|pmod |SELECT pmod(10, 3); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'pmod'.\nDid you mean 'pow'?\n | +|posexplode |SELECT posexplode(array(10,20)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 10")\n | +|posexplode_outer |SELECT posexplode_outer(array(10,20)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 10")\n | +|position |SELECT position('bar', 'foobarbar'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Position function must include IN keyword")\n | +|positive |SELECT positive(1); |PASSED |OK |std_err: Error during planning: Invalid function 'positive'.\nDid you mean 'position'?\n | +|pow |SELECT pow(2, 3); |PASSED |OK |OK | +|power |SELECT power(2, 3); |PASSED |OK |OK | +|printf |SELECT printf("Hello World %d %s", 100, "days"); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'printf'.\nDid you mean 'asinh'?\n | +|quarter |SELECT quarter('2016-08-31'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'quarter'.\nDid you mean 'flatten'?\n | +|radians |SELECT radians(180); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|raise_error |SELECT raise_error('custom error message'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'raise_error'.\nDid you mean 'make_array'?\n | +|rand |SELECT rand(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'rand'.\nDid you mean 'tan'?\n | +|randn |SELECT randn(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'randn'.\nDid you mean 'random'?\n | +|random |SELECT random(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|rank |SELECT a, b, rank(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: No function matches the given name and argument types 'RANK(Int32)'. You might need to add explicit type casts.\n\tCandidate functions:\n\tRANK()\n | +|reduce |SELECT reduce(array(1, 2, 3), 0, (acc, x) -> acc + x); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|reflect |SELECT reflect('java.util.UUID', 'randomUUID'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'reflect'.\nDid you mean 'replace'?\n | +|regexp |SELECT regexp('%SystemDrive%\Users\John', '%SystemDrive%\\Users.*'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: TokenizerError("unsupported escape char: '\\U'")\n | +|regexp_count |SELECT regexp_count('Steven Jones and Stephen Smith are the best players', 'Ste(v|ph)en'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'regexp_count'.\nDid you mean 'REGR_COUNT'?\n | +|regexp_extract |SELECT regexp_extract('100-200', '(\\d+)-(\\d+)', 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'regexp_extract'.\nDid you mean 'regexp_match'?\n | +|regexp_extract_all |SELECT regexp_extract_all('100-200, 300-400', '(\\d+)-(\\d+)', 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'regexp_extract_all'.\nDid you mean 'regexp_match'?\n | +|regexp_instr |SELECT regexp_instr('user@spark.apache.org', '@[^.]*'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'regexp_instr'.\nDid you mean 'regexp_like'?\n | +|regexp_like |SELECT regexp_like('%SystemDrive%\Users\John', '%SystemDrive%\\Users.*'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: TokenizerError("unsupported escape char: '\\U'")\n | +|regexp_replace |SELECT regexp_replace('100-200', '(\\d+)', 'num'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|regexp_substr |SELECT regexp_substr('Steven Jones and Stephen Smith are the best players', 'Ste(v|ph)en'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'regexp_substr'.\nDid you mean 'regexp_like'?\n | +|regr_avgx |SELECT regr_avgx(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|regr_avgy |SELECT regr_avgy(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|regr_count |SELECT regr_count(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|regr_intercept |SELECT regr_intercept(y, x) FROM VALUES (1,1), (2,2), (3,3) AS tab(y, x); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|regr_r2 |SELECT regr_r2(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|regr_slope |SELECT regr_slope(y, x) FROM VALUES (1,1), (2,2), (3,3) AS tab(y, x); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|regr_sxx |SELECT regr_sxx(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|regr_sxy |SELECT regr_sxy(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|regr_syy |SELECT regr_syy(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|repeat |SELECT repeat('123', 2); |PASSED |OK |OK | +|replace |SELECT replace('ABCabc', 'abc', 'DEF'); |PASSED |OK |OK | +|reverse |SELECT reverse('Spark SQL'); |PASSED |OK |OK | +|right |SELECT right('Spark SQL', 3); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|rint |SELECT rint(12.3456); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'rint'.\nDid you mean 'sin'?\n | +|rlike |SELECT rlike('%SystemDrive%\Users\John', '%SystemDrive%\\Users.*'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: TokenizerError("unsupported escape char: '\\U'")\n | +|round |SELECT round(2.5, 0); |PASSED |OK |OK | +|row_number |SELECT a, b, row_number() OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|rpad |SELECT rpad('hi', 5, '??'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|rtrim |SELECT rtrim(' SparkSQL '); |PASSED |OK |OK | +|schema_of_csv |SELECT schema_of_csv('1,abc'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'schema_of_csv'.\nDid you mean 'concat_ws'?\n | +|schema_of_json |SELECT schema_of_json('[{"col":0}]'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'schema_of_json'.\nDid you mean 'concat_ws'?\n | +|sec |SELECT sec(0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'sec'.\nDid you mean 'sin'?\n | +|second |SELECT second('2009-07-30 12:58:59'); |FAILED |Failed on native side: found CometNativeException |std_err: Error during planning: Invalid function 'second'.\nDid you mean 'decode'?\n | +|sentences |SELECT sentences('Hi there! Good morning.'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'sentences'.\nDid you mean 'degrees'?\n | +|sequence |SELECT sequence(1, 5); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'sequence'.\nDid you mean 'coalesce'?\n | +|sha |SELECT sha('Spark'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'sha'.\nDid you mean 'chr'?\n | +|sha1 |SELECT sha1('Spark'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'sha1'.\nDid you mean 'sha512'?\n | +|sha2 |SELECT sha2('Spark', 256); |PASSED |OK |std_err: Error during planning: Invalid function 'sha2'.\nDid you mean 'sha224'?\n | +|shiftleft |SELECT shiftleft(2, 1); |PASSED |OK |std_err: Error during planning: Invalid function 'shiftleft'.\nDid you mean 'left'?\n | +|shiftright |SELECT shiftright(4, 1); |PASSED |OK |std_err: Error during planning: Invalid function 'shiftright'.\nDid you mean 'right'?\n | +|shiftrightunsigned |SELECT shiftrightunsigned(4, 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'shiftrightunsigned'.\nDid you mean 'list_union'?\n | +|shuffle |SELECT shuffle(array(1, 20, 3, 5)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|sign |SELECT sign(40); |PASSED |OK |std_err: Error during planning: Invalid function 'sign'.\nDid you mean 'sin'?\n | +|signum |SELECT signum(40); |PASSED |OK |OK | +|sin |SELECT sin(0); |PASSED |OK |OK | +|sinh |SELECT sinh(0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|size |SELECT size(array('b', 'd', 'c', 'a')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 'b'")\n | +|skewness |SELECT skewness(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'skewness'.\nDid you mean 'degrees'?\n | +|slice |SELECT slice(array(1, 2, 3, 4), 2, 2); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|smallint | |SKIPPED|No examples found in spark.sessionState.functionRegistry | | +|some |SELECT some(col) FROM VALUES (true), (false), (false) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'some'.\nDid you mean 'SUM'?\n | +|sort_array |SELECT sort_array(array('b', 'd', null, 'c', 'a'), true); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 'b'")\n | +|soundex |SELECT soundex('Miller'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'soundex'.\nDid you mean 'round'?\n | +|space |SELECT concat(space(2), '1'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'space'.\nDid you mean 'lpad'?\n | +|spark_partition_id |SELECT spark_partition_id(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'spark_partition_id'.\nDid you mean 'list_position'?\n | +|split |SELECT split('oneAtwoBthreeC', '[ABC]'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'split'.\nDid you mean 'sqrt'?\n | +|split_part |SELECT split_part('11.12.13', '.', 3); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|sqrt |SELECT sqrt(4); |PASSED |OK |OK | +|stack |SELECT stack(2, 1, 2, 3); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'stack'.\nDid you mean 'atanh'?\n | +|startswith |SELECT startswith('Spark SQL', 'Spark'); |FAILED |Failed on native side: found CometNativeException |std_err: Error during planning: Invalid function 'startswith'.\nDid you mean 'starts_with'?\n | +|std |SELECT std(col) FROM VALUES (1), (2), (3) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'std'.\nDid you mean 'gcd'?\n | +|stddev |SELECT stddev(col) FROM VALUES (1), (2), (3) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|stddev_pop |SELECT stddev_pop(col) FROM VALUES (1), (2), (3) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|stddev_samp |SELECT stddev_samp(col) FROM VALUES (1), (2), (3) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|str_to_map |SELECT str_to_map('a:1,b:2,c:3', ',', ':'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'str_to_map'.\nDid you mean 'strpos'?\n | +|string | |SKIPPED|No examples found in spark.sessionState.functionRegistry | | +|struct |SELECT struct(1, 2, 3); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|substr |SELECT substr('Spark SQL', 5); |FAILED |Failed on native side: found CometNativeException | | +|substring |SELECT substring('Spark SQL', 5); |FAILED |Failed on native side: found CometNativeException |std_err: This feature is not implemented: Unsupported ast node in sqltorel: Substring { expr: Value(SingleQuotedString("Spark SQL")), substring_from: Some(Value(Number("5", false))), substring_for: None, special: true }\n | +|substring_index |SELECT substring_index('www.apache.org', '.', 2); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|sum |SELECT sum(col) FROM VALUES (5), (10), (15) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|tan |SELECT tan(0); |PASSED |OK |OK | +|tanh |SELECT tanh(0); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|timestamp | |SKIPPED|No examples found in spark.sessionState.functionRegistry | | +|timestamp_micros |SELECT timestamp_micros(1230219000123123); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'timestamp_micros'.\nDid you mean 'to_timestamp_micros'?\n | +|timestamp_millis |SELECT timestamp_millis(1230219000123); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'timestamp_millis'.\nDid you mean 'to_timestamp_millis'?\n | +|timestamp_seconds |SELECT timestamp_seconds(1230219000); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'timestamp_seconds'.\nDid you mean 'to_timestamp_seconds'?\n | +|tinyint | |SKIPPED|No examples found in spark.sessionState.functionRegistry | | +|to_binary |SELECT to_binary('abc', 'utf-8'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'to_binary'.\nDid you mean 'to_char'?\n | +|to_char |SELECT to_char(454, '999'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: No function matches the given name and argument types 'to_char(Int64, Utf8)'. You might need to add explicit type casts.\n\tCandidate functions:\n\tto_char(Date32, Utf8)\n\tto_char(Date64, Utf8)\n\tto_char(Time32(Millisecond), Utf8)\n\tto_char(Time32(Second), Utf8)\n\tto_char(Time64(Microsecond), Utf8)\n\tto_char(Time64(Nanosecond), Utf8)\n\tto_char(Timestamp(Second, None), Utf8)\n\tto_char(Timestamp(Second, Some("+TZ")), Utf8)\n\tto_char(Timestamp(Millisecond, None), Utf8)\n\tto_char(Timestamp(Millisecond, Some("+TZ")), Utf8)\n\tto_char(Timestamp(Microsecond, None), Utf8)\n\tto_char(Timestamp(Microsecond, Some("+TZ")), Utf8)\n\tto_char(Timestamp(Nanosecond, None), Utf8)\n\tto_char(Timestamp(Nanosecond, Some("+TZ")), Utf8)\n\tto_char(Duration(Second), Utf8)\n\tto_char(Duration(Millisecond), Utf8)\n\tto_char(Duration(Microsecond), Utf8)\n\tto_char(Duration(Nanosecond), Utf8)\n| +|to_csv |SELECT to_csv(named_struct('a', 1, 'b', 2)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'to_csv'.\nDid you mean 'to_hex'?\n | +|to_date |SELECT to_date('2009-07-30 04:17:52'); |PASSED |OK |OK | +|to_json |SELECT to_json(named_struct('a', 1, 'b', 2)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'to_json'.\nDid you mean 'to_hex'?\n | +|to_number |SELECT to_number('454', '999'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'to_number'.\nDid you mean 'to_char'?\n | +|to_timestamp |SELECT to_timestamp('2016-12-31 00:12:00'); |PASSED |OK |OK | +|to_timestamp_ltz |SELECT to_timestamp_ltz('2016-12-31 00:12:00'); |PASSED |OK |std_err: Error during planning: Invalid function 'to_timestamp_ltz'.\nDid you mean 'to_timestamp'?\n | +|to_timestamp_ntz |SELECT to_timestamp_ntz('2016-12-31 00:12:00'); |FAILED |Failed on native side: found CometNativeException |std_err: Error during planning: Invalid function 'to_timestamp_ntz'.\nDid you mean 'to_timestamp_nanos'?\n | +|to_unix_timestamp |SELECT to_unix_timestamp('2016-04-08', 'yyyy-MM-dd'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'to_unix_timestamp'.\nDid you mean 'to_timestamp'?\n | +|to_utc_timestamp |SELECT to_utc_timestamp('2016-08-31', 'Asia/Seoul'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'to_utc_timestamp'.\nDid you mean 'to_timestamp'?\n | +|transform |SELECT transform(array(1, 2, 3), x -> x + 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|transform_keys |SELECT transform_keys(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|transform_values |SELECT transform_values(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> v + 1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|translate |SELECT translate('AaBbCc', 'abc', '123'); |PASSED |OK |OK | +|trim |SELECT trim(' SparkSQL '); |PASSED |OK |OK | +|trunc |SELECT trunc('2019-08-04', 'week'); |FAILED |Failed on native side: found CometNativeException |std_err: Error during planning: No function matches the given name and argument types 'trunc(Utf8, Utf8)'. You might need to add explicit type casts.\n\tCandidate functions:\n\ttrunc(Float32, Int64)\n\ttrunc(Float64, Int64)\n\ttrunc(Float64)\n\ttrunc(Float32)\n | +|try_add |SELECT try_add(1, 2); |PASSED |OK |std_err: Error during planning: Invalid function 'try_add'.\nDid you mean 'rpad'?\n | +|try_avg |SELECT try_avg(col) FROM VALUES (1), (2), (3) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'try_avg'.\nDid you mean 'AVG'?\n | +|try_divide |SELECT try_divide(3, 2); |PASSED |OK |std_err: Error during planning: Invalid function 'try_divide'.\nDid you mean 'to_date'?\n | +|try_element_at |SELECT try_element_at(array(1, 2, 3), 2); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|try_multiply |SELECT try_multiply(2, 3); |PASSED |OK |std_err: Error during planning: Invalid function 'try_multiply'.\nDid you mean 'array_union'?\n | +|try_subtract |SELECT try_subtract(2, 1); |PASSED |OK |std_err: Error during planning: Invalid function 'try_subtract'.\nDid you mean 'array_extract'?\n | +|try_sum |SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'try_sum'.\nDid you mean 'trim'?\n | +|try_to_binary |SELECT try_to_binary('abc', 'utf-8'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'try_to_binary'.\nDid you mean 'string_to_array'?\n | +|try_to_number |SELECT try_to_number('454', '999'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'try_to_number'.\nDid you mean 'array_to_string'?\n | +|try_to_timestamp |SELECT try_to_timestamp('2016-12-31 00:12:00'); |PASSED |OK |std_err: Error during planning: Invalid function 'try_to_timestamp'.\nDid you mean 'to_timestamp'?\n | +|typeof |SELECT typeof(1); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'typeof'.\nDid you mean 'repeat'?\n | +|ucase |SELECT ucase('SparkSql'); |PASSED |OK |std_err: Error during planning: Invalid function 'ucase'.\nDid you mean 'acos'?\n | +|unbase64 |SELECT unbase64('U3BhcmsgU1FM'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'unbase64'.\nDid you mean 'sha256'?\n | +|unhex |SELECT decode(unhex('537061726B2053514C'), 'UTF-8'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'unhex'.\nDid you mean 'upper'?\n | +|unix_date |SELECT unix_date(DATE("1970-01-02")); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'unix_date'.\nDid you mean 'to_date'?\n | +|unix_micros |SELECT unix_micros(TIMESTAMP('1970-01-01 00:00:01Z')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'unix_micros'.\nDid you mean 'initcap'?\n | +|unix_millis |SELECT unix_millis(TIMESTAMP('1970-01-01 00:00:01Z')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'unix_millis'.\nDid you mean 'nullif'?\n | +|unix_seconds |SELECT unix_seconds(TIMESTAMP('1970-01-01 00:00:01Z')); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'unix_seconds'.\nDid you mean 'decode'?\n | +|unix_timestamp |SELECT unix_timestamp(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'unix_timestamp'.\nDid you mean 'to_timestamp'?\n | +|upper |SELECT upper('SparkSql'); |PASSED |OK |OK | +|url_decode |SELECT url_decode('https%3A%2F%2Fspark.apache.org'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'url_decode'.\nDid you mean 'decode'?\n | +|url_encode |SELECT url_encode('https://spark.apache.org'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'url_encode'.\nDid you mean 'encode'?\n | +|user |SELECT user(); |PASSED |OK |std_err: SQL error: ParserError("Expected end of statement, found: (")\n | +|uuid |SELECT uuid(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|var_pop |SELECT var_pop(col) FROM VALUES (1), (2), (3) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|var_samp |SELECT var_samp(col) FROM VALUES (1), (2), (3) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback | | +|variance |SELECT variance(col) FROM VALUES (1), (2), (3) AS tab(col); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_out: DataFusion CLI v37.1.0\nError: Error during planning: Invalid function 'variance'.\nDid you mean 'range'?\n | +|version |SELECT version(); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'version'.\nDid you mean 'sin'?\n | +|weekday |SELECT weekday('2009-07-30'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'weekday'.\nDid you mean 'today'?\n | +|weekofyear |SELECT weekofyear('2008-02-20'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'weekofyear'.\nDid you mean 'repeat'?\n | +|when |SELECT CASE WHEN 1 > 0 THEN 1 WHEN 2 > 0 THEN 2.0 ELSE 1.2 END; |PASSED |OK |OK | +|width_bucket |SELECT width_bucket(5.3, 0.2, 10.6, 5); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'width_bucket'.\nDid you mean 'list_cat'?\n | +|xpath |SELECT xpath('b1b2b3c1c2','a/b/text()'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'xpath'.\nDid you mean 'tanh'?\n | +|xpath_boolean |SELECT xpath_boolean('1','a/b'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'xpath_boolean'.\nDid you mean 'date_format'?\n | +|xpath_double |SELECT xpath_double('12', 'sum(a/b)'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'xpath_double'.\nDid you mean 'date_format'?\n | +|xpath_float |SELECT xpath_float('12', 'sum(a/b)'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'xpath_float'.\nDid you mean 'date_format'?\n | +|xpath_int |SELECT xpath_int('12', 'sum(a/b)'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'xpath_int'.\nDid you mean 'date_bin'?\n | +|xpath_long |SELECT xpath_long('12', 'sum(a/b)'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'xpath_long'.\nDid you mean 'date_bin'?\n | +|xpath_number |SELECT xpath_number('12', 'sum(a/b)'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'xpath_number'.\nDid you mean 'date_bin'?\n | +|xpath_short |SELECT xpath_short('12', 'sum(a/b)'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'xpath_short'.\nDid you mean 'list_sort'?\n | +|xpath_string |SELECT xpath_string('bcc','a/c'); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: Error during planning: Invalid function 'xpath_string'.\nDid you mean 'date_trunc'?\n | +|xxhash64 |SELECT xxhash64('Spark', array(123), 2); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 123")\n | +|year |SELECT year('2016-07-30'); |PASSED |OK |std_err: Error during planning: Invalid function 'year'.\nDid you mean 'VAR'?\n | +|zip_with |SELECT zip_with(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x)); |FAILED |Unsupported: Expected only Comet native operators but found Spark fallback |std_err: SQL error: ParserError("Expected SELECT, VALUES, or a subquery in the query body, found: 1")\n | +|| |SELECT 3 | 5; |PASSED |OK |OK | +|~ |SELECT ~ 0; |FAILED |Failed on native side: found CometNativeException |std_err: SQL error: ParserError("Expected an expression:, found: ~")\n | ++---------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------+-------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ diff --git a/pom.xml b/pom.xml index 5b054f001..59e0569ff 100644 --- a/pom.xml +++ b/pom.xml @@ -899,7 +899,7 @@ under the License. tpcds-kit/** tpcds-sf-1/** tpch/** - doc/*.txt + docs/*.txt diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionCoverageSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionCoverageSuite.scala index 2011dcd4b..26c9c8fec 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionCoverageSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionCoverageSuite.scala @@ -23,32 +23,37 @@ import java.nio.charset.StandardCharsets import java.nio.file.{Files, Paths} import scala.collection.mutable +import scala.sys.process._ import org.scalatest.Ignore import org.scalatest.exceptions.TestFailedException +import org.apache.hadoop.fs.Path import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.comet.CoverageResultStatus.CoverageResultStatus + /** * Manual test to calculate Spark builtin expressions coverage support by the Comet * - * The test will update files doc/spark_builtin_expr_coverage.txt, - * doc/spark_builtin_expr_coverage_agg.txt + * The test will update files docs/spark_builtin_expr_coverage.txt */ @Ignore class CometExpressionCoverageSuite extends CometTestBase with AdaptiveSparkPlanHelper { import testImplicits._ - private val rawCoverageFilePath = "doc/spark_builtin_expr_coverage.txt" - private val aggCoverageFilePath = "doc/spark_builtin_expr_coverage_agg.txt" + private val projectDocFolder = "docs" + private val rawCoverageFilePath = s"$projectDocFolder/spark_builtin_expr_coverage.txt" + private val DATAFUSIONCLI_PATH_ENV_VAR = "DATAFUSIONCLI_PATH" - test("Test Spark builtin expressions coverage") { - val queryPattern = """(?i)SELECT (.+?);""".r - val valuesPattern = """(?i)FROM VALUES(.+?);""".r - val selectPattern = """(i?)SELECT(.+?)FROM""".r - val builtinExamplesMap = spark.sessionState.functionRegistry + private val queryPattern = """(?i)SELECT (.+?);""".r + private val valuesPattern = """(?i)FROM VALUES(.+?);""".r + private val selectPattern = """(i?)SELECT(.+?)FROM""".r + + def getExamples(): Map[String, List[String]] = + spark.sessionState.functionRegistry .listFunction() .map(spark.sessionState.catalog.lookupFunctionInfo(_)) .filter(_.getSource.toLowerCase == "built-in") @@ -61,12 +66,22 @@ class CometExpressionCoverageSuite extends CometTestBase with AdaptiveSparkPlanH }) .toMap + /** + * Manual test to calculate Spark builtin expressions coverage support by the Comet + * + * The test will update files doc/spark_builtin_expr_coverage.txt, + * doc/spark_builtin_expr_coverage_agg.txt + */ + test("Test Spark builtin expressions coverage") { + val builtinExamplesMap = getExamples() + // key - function name // value - list of result shows if function supported by Comet val resultsMap = new mutable.HashMap[String, CoverageResult]() builtinExamplesMap.foreach { case (funcName, q :: _) => + var dfMessage: Option[String] = None val queryResult = try { // Example with predefined values @@ -78,12 +93,15 @@ class CometExpressionCoverageSuite extends CometTestBase with AdaptiveSparkPlanH val values = valuesPattern.findFirstMatchIn(q).map(_.group(0)) (select, values) match { case (Some(s), Some(v)) => - testSingleLineQuery(s"select * $v", s"$s tbl") + withTempDir { dir => + val path = new Path(dir.toURI.toString).toUri.getPath + spark.sql(s"select * $v").repartition(1).write.mode("overwrite").parquet(path) + dfMessage = runDatafusionCli(s"""$s '$path/*.parquet'""") + } + testSingleLineQuery(s"select * $v", s"$s tbl") case _ => - resultsMap.put( - funcName, - CoverageResult("FAILED", Seq((q, "Cannot parse properly")))) + sys.error("Cannot parse properly") } } else { // Process the simple example like `SELECT cos(0);` @@ -93,51 +111,107 @@ class CometExpressionCoverageSuite extends CometTestBase with AdaptiveSparkPlanH // // ConstantFolding is a operator optimization rule in Catalyst that replaces expressions // that can be statically evaluated with their equivalent literal values. + dfMessage = runDatafusionCli(q) testSingleLineQuery( "select 'dummy' x", s"${q.dropRight(1)}, x from tbl", excludedOptimizerRules = Some("org.apache.spark.sql.catalyst.optimizer.ConstantFolding")) } - CoverageResult(CoverageResultStatus.Passed.toString, Seq((q, "OK"))) + CoverageResult( + q, + CoverageResultStatus.Passed, + CoverageResultDetails( + cometMessage = "OK", + datafusionMessage = dfMessage.getOrElse("OK"))) + } catch { case e: TestFailedException - if e.message.getOrElse("").contains("Expected only Comet native operators") => - CoverageResult(CoverageResultStatus.Failed.toString, Seq((q, "Unsupported"))) + if e.getMessage.contains("Expected only Comet native operators") => + CoverageResult( + q, + CoverageResultStatus.Failed, + CoverageResultDetails( + cometMessage = + "Unsupported: Expected only Comet native operators but found Spark fallback", + datafusionMessage = dfMessage.getOrElse(""))) + case e if e.getMessage.contains("CometNativeException") => CoverageResult( - CoverageResultStatus.Failed.toString, - Seq((q, "Failed on native side"))) - case _ => + q, + CoverageResultStatus.Failed, + CoverageResultDetails( + cometMessage = "Failed on native side: found CometNativeException", + datafusionMessage = dfMessage.getOrElse(""))) + + case e => CoverageResult( - CoverageResultStatus.Failed.toString, - Seq((q, "Failed on something else. Check query manually"))) + q, + CoverageResultStatus.Failed, + CoverageResultDetails( + cometMessage = e.getMessage, + datafusionMessage = dfMessage.getOrElse(""))) } resultsMap.put(funcName, queryResult) + // Function with no examples case (funcName, List()) => resultsMap.put( funcName, CoverageResult( - CoverageResultStatus.Skipped.toString, - Seq(("", "No examples found in spark.sessionState.functionRegistry")))) + "", + CoverageResultStatus.Skipped, + CoverageResultDetails( + cometMessage = "No examples found in spark.sessionState.functionRegistry", + datafusionMessage = ""))) } - // TODO: convert results into HTML + // TODO: convert results into HTML or .md file resultsMap.toSeq.toDF("name", "details").createOrReplaceTempView("t") - val str_agg = showString( + + val str = showString( spark.sql( - "select result, d._2 as details, count(1) cnt from (select name, t.details.result, explode_outer(t.details.details) as d from t) group by 1, 2 order by 1"), + "select name, details.query, details.result, details.details.cometMessage, details.details.datafusionMessage from t order by 1"), 1000, 0) - Files.write(Paths.get(aggCoverageFilePath), str_agg.getBytes(StandardCharsets.UTF_8)) - - val str = showString(spark.sql("select * from t order by 1"), 1000, 0) Files.write(Paths.get(rawCoverageFilePath), str.getBytes(StandardCharsets.UTF_8)) } -} -case class CoverageResult(result: String, details: Seq[(String, String)]) + // Returns execution error, None means successful execution + private def runDatafusionCli(sql: String): Option[String] = { + + val datafusionCliPath = sys.env.getOrElse( + DATAFUSIONCLI_PATH_ENV_VAR, + return Some(s"$DATAFUSIONCLI_PATH_ENV_VAR env variable not set")) + + val tempFilePath = Files.createTempFile("temp-", ".sql") + val stdout = new StringBuilder + val stderr = new StringBuilder + try { + Files.write(tempFilePath, sql.getBytes) + + val command = s"""$datafusionCliPath/datafusion-cli -f $tempFilePath""" + command.!( + ProcessLogger( + out => stdout.append(out).append("\n"), // stdout + err => stderr.append(err).append("\n") // stderr + )) + } finally { + Files.delete(tempFilePath) + } + + val err = stderr.toString + val out = stdout.toString + + if (err.nonEmpty) + return Some(s"std_err: $err") + + if (out.toLowerCase.contains("error")) + return Some(s"std_out: $out") + + None + } +} object CoverageResultStatus extends Enumeration { type CoverageResultStatus = Value @@ -146,3 +220,10 @@ object CoverageResultStatus extends Enumeration { val Passed: Value = Value("PASSED") val Skipped: Value = Value("SKIPPED") } + +case class CoverageResult( + query: String, + result: CoverageResultStatus, + details: CoverageResultDetails) + +case class CoverageResultDetails(cometMessage: String, datafusionMessage: String) From de8fe455be6e875535aa5c974f76e902636fac9e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 20 May 2024 20:02:15 -0600 Subject: [PATCH 04/14] fix: Enable cast string to int tests and fix compatibility issue (#453) * simplify cast string to int logic and use untrimmed string in error messages * remove state enum --- .../execution/datafusion/expressions/cast.rs | 57 ++++--------------- docs/source/user-guide/compatibility.md | 8 +-- .../apache/comet/expressions/CometCast.scala | 2 +- .../org/apache/comet/CometCastSuite.scala | 8 +-- 4 files changed, 20 insertions(+), 55 deletions(-) diff --git a/core/src/execution/datafusion/expressions/cast.rs b/core/src/execution/datafusion/expressions/cast.rs index 35ab23a76..f68732fb1 100644 --- a/core/src/execution/datafusion/expressions/cast.rs +++ b/core/src/execution/datafusion/expressions/cast.rs @@ -82,7 +82,7 @@ macro_rules! cast_utf8_to_int { for i in 0..len { if $array.is_null(i) { cast_array.append_null() - } else if let Some(cast_value) = $cast_method($array.value(i).trim(), $eval_mode)? { + } else if let Some(cast_value) = $cast_method($array.value(i), $eval_mode)? { cast_array.append_value(cast_value); } else { cast_array.append_null() @@ -1010,14 +1010,6 @@ fn cast_string_to_int_with_range_check( } } -#[derive(PartialEq)] -enum State { - SkipLeadingWhiteSpace, - SkipTrailingWhiteSpace, - ParseSignAndDigits, - ParseFractionalDigits, -} - /// Equivalent to /// - org.apache.spark.unsafe.types.UTF8String.toInt(IntWrapper intWrapper, boolean allowDecimal) /// - org.apache.spark.unsafe.types.UTF8String.toLong(LongWrapper longWrapper, boolean allowDecimal) @@ -1029,34 +1021,22 @@ fn do_cast_string_to_int< type_name: &str, min_value: T, ) -> CometResult> { - let len = str.len(); - if str.is_empty() { + let trimmed_str = str.trim(); + if trimmed_str.is_empty() { return none_or_err(eval_mode, type_name, str); } - + let len = trimmed_str.len(); let mut result: T = T::zero(); let mut negative = false; let radix = T::from(10); let stop_value = min_value / radix; - let mut state = State::SkipLeadingWhiteSpace; - let mut parsed_sign = false; - - for (i, ch) in str.char_indices() { - // skip leading whitespace - if state == State::SkipLeadingWhiteSpace { - if ch.is_whitespace() { - // consume this char - continue; - } - // change state and fall through to next section - state = State::ParseSignAndDigits; - } + let mut parse_sign_and_digits = true; - if state == State::ParseSignAndDigits { - if !parsed_sign { + for (i, ch) in trimmed_str.char_indices() { + if parse_sign_and_digits { + if i == 0 { negative = ch == '-'; let positive = ch == '+'; - parsed_sign = true; if negative || positive { if i + 1 == len { // input string is just "+" or "-" @@ -1070,7 +1050,7 @@ fn do_cast_string_to_int< if ch == '.' { if eval_mode == EvalMode::Legacy { // truncate decimal in legacy mode - state = State::ParseFractionalDigits; + parse_sign_and_digits = false; continue; } else { return none_or_err(eval_mode, type_name, str); @@ -1102,27 +1082,12 @@ fn do_cast_string_to_int< return none_or_err(eval_mode, type_name, str); } } - } - - if state == State::ParseFractionalDigits { - // This is the case when we've encountered a decimal separator. The fractional - // part will not change the number, but we will verify that the fractional part - // is well-formed. - if ch.is_whitespace() { - // finished parsing fractional digits, now need to skip trailing whitespace - state = State::SkipTrailingWhiteSpace; - // consume this char - continue; - } + } else { + // make sure fractional digits are valid digits but ignore them if !ch.is_ascii_digit() { return none_or_err(eval_mode, type_name, str); } } - - // skip trailing whitespace - if state == State::SkipTrailingWhiteSpace && !ch.is_whitespace() { - return none_or_err(eval_mode, type_name, str); - } } if !negative { diff --git a/docs/source/user-guide/compatibility.md b/docs/source/user-guide/compatibility.md index 278edb848..a4ed9289f 100644 --- a/docs/source/user-guide/compatibility.md +++ b/docs/source/user-guide/compatibility.md @@ -110,6 +110,10 @@ The following cast operations are generally compatible with Spark except for the | decimal | float | | | decimal | double | | | string | boolean | | +| string | byte | | +| string | short | | +| string | integer | | +| string | long | | | string | binary | | | date | string | | | timestamp | long | | @@ -125,10 +129,6 @@ The following cast operations are not compatible with Spark for all inputs and a |-|-|-| | integer | decimal | No overflow check | | long | decimal | No overflow check | -| string | byte | Not all invalid inputs are detected | -| string | short | Not all invalid inputs are detected | -| string | integer | Not all invalid inputs are detected | -| string | long | Not all invalid inputs are detected | | string | timestamp | Not all valid formats are supported | | binary | string | Only works for binary data representing valid UTF-8 strings | diff --git a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala index 9c3695ba5..795bdb428 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -108,7 +108,7 @@ object CometCast { Compatible() case DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType => - Incompatible(Some("Not all invalid inputs are detected")) + Compatible() case DataTypes.BinaryType => Compatible() case DataTypes.FloatType | DataTypes.DoubleType => diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index ea3355d05..8caba14c6 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -519,28 +519,28 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { "9223372036854775808" // Long.MaxValue + 1 ) - ignore("cast StringType to ByteType") { + test("cast StringType to ByteType") { // test with hand-picked values castTest(castStringToIntegralInputs.toDF("a"), DataTypes.ByteType) // fuzz test castTest(gen.generateStrings(dataSize, numericPattern, 4).toDF("a"), DataTypes.ByteType) } - ignore("cast StringType to ShortType") { + test("cast StringType to ShortType") { // test with hand-picked values castTest(castStringToIntegralInputs.toDF("a"), DataTypes.ShortType) // fuzz test castTest(gen.generateStrings(dataSize, numericPattern, 5).toDF("a"), DataTypes.ShortType) } - ignore("cast StringType to IntegerType") { + test("cast StringType to IntegerType") { // test with hand-picked values castTest(castStringToIntegralInputs.toDF("a"), DataTypes.IntegerType) // fuzz test castTest(gen.generateStrings(dataSize, numericPattern, 8).toDF("a"), DataTypes.IntegerType) } - ignore("cast StringType to LongType") { + test("cast StringType to LongType") { // test with hand-picked values castTest(castStringToIntegralInputs.toDF("a"), DataTypes.LongType) // fuzz test From 7b0a7e07ad36b5f4e0b19bc8231deaa637ac03c7 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 21 May 2024 09:11:21 -0700 Subject: [PATCH 05/14] feat: Supports UUID column (#395) * fix uuid * address comments --------- Co-authored-by: Huaxin Gao --- .../org/apache/comet/parquet/ColumnReader.java | 14 +++++++++++--- .../apache/comet/vector/CometDecodedVector.java | 7 +++++++ .../apache/comet/vector/CometDictionaryVector.java | 9 +++++---- .../org/apache/comet/vector/CometPlainVector.java | 13 +++++++++++-- 4 files changed, 34 insertions(+), 9 deletions(-) diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java index 7e45f4f9a..46fd87f6b 100644 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -41,6 +41,7 @@ import org.apache.parquet.column.page.DataPageV2; import org.apache.parquet.column.page.DictionaryPage; import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.spark.sql.types.DataType; import org.apache.comet.CometConf; @@ -199,6 +200,11 @@ public CometDecodedVector loadVector() { currentVector.close(); } + LogicalTypeAnnotation logicalTypeAnnotation = + descriptor.getPrimitiveType().getLogicalTypeAnnotation(); + boolean isUuid = + logicalTypeAnnotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation; + long[] addresses = Native.currentBatch(nativeHandle); try (ArrowArray array = ArrowArray.wrap(addresses[0]); @@ -206,7 +212,7 @@ public CometDecodedVector loadVector() { FieldVector vector = Data.importVector(ALLOCATOR, array, schema, dictionaryProvider); DictionaryEncoding dictionaryEncoding = vector.getField().getDictionary(); - CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128); + CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128, isUuid); // Update whether the current vector contains any null values. This is used in the following // batch(s) to determine whether we can skip loading the native vector. @@ -229,12 +235,14 @@ public CometDecodedVector loadVector() { // initialized yet. Dictionary arrowDictionary = dictionaryProvider.lookup(dictionaryEncoding.getId()); CometPlainVector dictionaryVector = - new CometPlainVector(arrowDictionary.getVector(), useDecimal128); + new CometPlainVector(arrowDictionary.getVector(), useDecimal128, isUuid); dictionary = new CometDictionary(dictionaryVector); } currentVector = - new CometDictionaryVector(cometVector, dictionary, dictionaryProvider, useDecimal128); + new CometDictionaryVector( + cometVector, dictionary, dictionaryProvider, useDecimal128, false, isUuid); + return currentVector; } } diff --git a/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java b/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java index 5ebe6923a..f699134f8 100644 --- a/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java @@ -38,13 +38,20 @@ public abstract class CometDecodedVector extends CometVector { private int numValues; private int validityByteCacheIndex = -1; private byte validityByteCache; + protected boolean isUuid; protected CometDecodedVector(ValueVector vector, Field valueField, boolean useDecimal128) { + this(vector, valueField, useDecimal128, false); + } + + protected CometDecodedVector( + ValueVector vector, Field valueField, boolean useDecimal128, boolean isUuid) { super(Utils.fromArrowField(valueField), useDecimal128); this.valueVector = vector; this.numNulls = valueVector.getNullCount(); this.numValues = valueVector.getValueCount(); this.hasNull = numNulls != 0; + this.isUuid = isUuid; } @Override diff --git a/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java b/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java index 2cd9c5d18..a74f4ff6b 100644 --- a/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java @@ -39,7 +39,7 @@ public CometDictionaryVector( CometDictionary values, DictionaryProvider provider, boolean useDecimal128) { - this(indices, values, provider, useDecimal128, false); + this(indices, values, provider, useDecimal128, false, false); } public CometDictionaryVector( @@ -47,8 +47,9 @@ public CometDictionaryVector( CometDictionary values, DictionaryProvider provider, boolean useDecimal128, - boolean isAlias) { - super(indices.valueVector, values.getValueVector().getField(), useDecimal128); + boolean isAlias, + boolean isUuid) { + super(indices.valueVector, values.getValueVector().getField(), useDecimal128, isUuid); Preconditions.checkArgument( indices.valueVector instanceof IntVector, "'indices' should be a IntVector"); this.values = values; @@ -130,6 +131,6 @@ public CometVector slice(int offset, int length) { // Set the alias flag to true so that the sliced vector will not close the dictionary vector. // Otherwise, if the dictionary is closed, the sliced vector will not be able to access the // dictionary. - return new CometDictionaryVector(sliced, values, provider, useDecimal128, true); + return new CometDictionaryVector(sliced, values, provider, useDecimal128, true, isUuid); } } diff --git a/common/src/main/java/org/apache/comet/vector/CometPlainVector.java b/common/src/main/java/org/apache/comet/vector/CometPlainVector.java index 521f1047c..e2a625f01 100644 --- a/common/src/main/java/org/apache/comet/vector/CometPlainVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometPlainVector.java @@ -39,7 +39,11 @@ public class CometPlainVector extends CometDecodedVector { private int booleanByteCacheIndex = -1; public CometPlainVector(ValueVector vector, boolean useDecimal128) { - super(vector, vector.getField(), useDecimal128); + this(vector, useDecimal128, false); + } + + public CometPlainVector(ValueVector vector, boolean useDecimal128, boolean isUuid) { + super(vector, vector.getField(), useDecimal128, isUuid); // NullType doesn't have data buffer. if (vector instanceof NullVector) { this.valueBufferAddress = -1; @@ -111,7 +115,12 @@ public UTF8String getUTF8String(int rowId) { byte[] result = new byte[length]; Platform.copyMemory( null, valueBufferAddress + offset, result, Platform.BYTE_ARRAY_OFFSET, length); - return UTF8String.fromBytes(result); + + if (!isUuid) { + return UTF8String.fromBytes(result); + } else { + return UTF8String.fromString(convertToUuid(result).toString()); + } } } From 6d23e2846c5ba0049bb7dc25955ab5d717953f9a Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 23 May 2024 08:20:27 -0700 Subject: [PATCH 06/14] feat: correlation support (#456) * feat: correlation support * fmt * remove un-used import * address comments * address comment --------- Co-authored-by: Huaxin Gao --- .../datafusion/expressions/correlation.rs | 256 ++++++++++++++++++ .../execution/datafusion/expressions/mod.rs | 1 + core/src/execution/datafusion/planner.rs | 13 + core/src/execution/proto/expr.proto | 8 + docs/source/user-guide/expressions.md | 1 + .../apache/comet/serde/QueryPlanSerde.scala | 23 +- .../comet/exec/CometAggregateSuite.scala | 151 +++++++++++ 7 files changed, 452 insertions(+), 1 deletion(-) create mode 100644 core/src/execution/datafusion/expressions/correlation.rs diff --git a/core/src/execution/datafusion/expressions/correlation.rs b/core/src/execution/datafusion/expressions/correlation.rs new file mode 100644 index 000000000..c83341e54 --- /dev/null +++ b/core/src/execution/datafusion/expressions/correlation.rs @@ -0,0 +1,256 @@ +// 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 arrow::compute::{and, filter, is_not_null}; + +use std::{any::Any, sync::Arc}; + +use crate::execution::datafusion::expressions::{ + covariance::CovarianceAccumulator, stats::StatsType, stddev::StddevAccumulator, + utils::down_cast_any_ref, +}; +use arrow::{ + array::ArrayRef, + datatypes::{DataType, Field}, +}; +use datafusion::logical_expr::Accumulator; +use datafusion_common::{Result, ScalarValue}; +use datafusion_physical_expr::{expressions::format_state_name, AggregateExpr, PhysicalExpr}; + +/// CORR aggregate expression +/// The implementation mostly is the same as the DataFusion's implementation. The reason +/// we have our own implementation is that DataFusion has UInt64 for state_field `count`, +/// while Spark has Double for count. Also we have added `null_on_divide_by_zero` +/// to be consistent with Spark's implementation. +#[derive(Debug)] +pub struct Correlation { + name: String, + expr1: Arc, + expr2: Arc, + null_on_divide_by_zero: bool, +} + +impl Correlation { + pub fn new( + expr1: Arc, + expr2: Arc, + name: impl Into, + data_type: DataType, + null_on_divide_by_zero: bool, + ) -> Self { + // the result of correlation just support FLOAT64 data type. + assert!(matches!(data_type, DataType::Float64)); + Self { + name: name.into(), + expr1, + expr2, + null_on_divide_by_zero, + } + } +} + +impl AggregateExpr for Correlation { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn field(&self) -> Result { + Ok(Field::new(&self.name, DataType::Float64, true)) + } + + fn create_accumulator(&self) -> Result> { + Ok(Box::new(CorrelationAccumulator::try_new( + self.null_on_divide_by_zero, + )?)) + } + + fn state_fields(&self) -> Result> { + Ok(vec![ + Field::new( + format_state_name(&self.name, "count"), + DataType::Float64, + true, + ), + Field::new( + format_state_name(&self.name, "mean1"), + DataType::Float64, + true, + ), + Field::new( + format_state_name(&self.name, "mean2"), + DataType::Float64, + true, + ), + Field::new( + format_state_name(&self.name, "algo_const"), + DataType::Float64, + true, + ), + Field::new( + format_state_name(&self.name, "m2_1"), + DataType::Float64, + true, + ), + Field::new( + format_state_name(&self.name, "m2_2"), + DataType::Float64, + true, + ), + ]) + } + + fn expressions(&self) -> Vec> { + vec![self.expr1.clone(), self.expr2.clone()] + } + + fn name(&self) -> &str { + &self.name + } +} + +impl PartialEq for Correlation { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::() + .map(|x| { + self.name == x.name + && self.expr1.eq(&x.expr1) + && self.expr2.eq(&x.expr2) + && self.null_on_divide_by_zero == x.null_on_divide_by_zero + }) + .unwrap_or(false) + } +} + +/// An accumulator to compute correlation +#[derive(Debug)] +pub struct CorrelationAccumulator { + covar: CovarianceAccumulator, + stddev1: StddevAccumulator, + stddev2: StddevAccumulator, + null_on_divide_by_zero: bool, +} + +impl CorrelationAccumulator { + /// Creates a new `CorrelationAccumulator` + pub fn try_new(null_on_divide_by_zero: bool) -> Result { + Ok(Self { + covar: CovarianceAccumulator::try_new(StatsType::Population)?, + stddev1: StddevAccumulator::try_new(StatsType::Population, null_on_divide_by_zero)?, + stddev2: StddevAccumulator::try_new(StatsType::Population, null_on_divide_by_zero)?, + null_on_divide_by_zero, + }) + } +} + +impl Accumulator for CorrelationAccumulator { + fn state(&mut self) -> Result> { + Ok(vec![ + ScalarValue::from(self.covar.get_count()), + ScalarValue::from(self.covar.get_mean1()), + ScalarValue::from(self.covar.get_mean2()), + ScalarValue::from(self.covar.get_algo_const()), + ScalarValue::from(self.stddev1.get_m2()), + ScalarValue::from(self.stddev2.get_m2()), + ]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values = if values[0].null_count() != 0 || values[1].null_count() != 0 { + let mask = and(&is_not_null(&values[0])?, &is_not_null(&values[1])?)?; + let values1 = filter(&values[0], &mask)?; + let values2 = filter(&values[1], &mask)?; + + vec![values1, values2] + } else { + values.to_vec() + }; + + if !values[0].is_empty() && !values[1].is_empty() { + self.covar.update_batch(&values)?; + self.stddev1.update_batch(&values[0..1])?; + self.stddev2.update_batch(&values[1..2])?; + } + + Ok(()) + } + + fn retract_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values = if values[0].null_count() != 0 || values[1].null_count() != 0 { + let mask = and(&is_not_null(&values[0])?, &is_not_null(&values[1])?)?; + let values1 = filter(&values[0], &mask)?; + let values2 = filter(&values[1], &mask)?; + + vec![values1, values2] + } else { + values.to_vec() + }; + + self.covar.retract_batch(&values)?; + self.stddev1.retract_batch(&values[0..1])?; + self.stddev2.retract_batch(&values[1..2])?; + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + let states_c = [ + states[0].clone(), + states[1].clone(), + states[2].clone(), + states[3].clone(), + ]; + let states_s1 = [states[0].clone(), states[1].clone(), states[4].clone()]; + let states_s2 = [states[0].clone(), states[2].clone(), states[5].clone()]; + + if states[0].len() > 0 && states[1].len() > 0 && states[2].len() > 0 { + self.covar.merge_batch(&states_c)?; + self.stddev1.merge_batch(&states_s1)?; + self.stddev2.merge_batch(&states_s2)?; + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + let covar = self.covar.evaluate()?; + let stddev1 = self.stddev1.evaluate()?; + let stddev2 = self.stddev2.evaluate()?; + + match (covar, stddev1, stddev2) { + ( + ScalarValue::Float64(Some(c)), + ScalarValue::Float64(Some(s1)), + ScalarValue::Float64(Some(s2)), + ) if s1 != 0.0 && s2 != 0.0 => Ok(ScalarValue::Float64(Some(c / (s1 * s2)))), + _ if self.null_on_divide_by_zero => Ok(ScalarValue::Float64(None)), + _ => { + if self.covar.get_count() == 1.0 { + return Ok(ScalarValue::Float64(Some(f64::NAN))); + } + Ok(ScalarValue::Float64(None)) + } + } + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) - std::mem::size_of_val(&self.covar) + self.covar.size() + - std::mem::size_of_val(&self.stddev1) + + self.stddev1.size() + - std::mem::size_of_val(&self.stddev2) + + self.stddev2.size() + } +} diff --git a/core/src/execution/datafusion/expressions/mod.rs b/core/src/execution/datafusion/expressions/mod.rs index 10cac1696..9db4b65b3 100644 --- a/core/src/execution/datafusion/expressions/mod.rs +++ b/core/src/execution/datafusion/expressions/mod.rs @@ -27,6 +27,7 @@ pub use normalize_nan::NormalizeNaNAndZero; pub mod avg; pub mod avg_decimal; pub mod bloom_filter_might_contain; +pub mod correlation; pub mod covariance; pub mod stats; pub mod stddev; diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index 59818857e..01d892381 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -67,6 +67,7 @@ use crate::{ bloom_filter_might_contain::BloomFilterMightContain, cast::{Cast, EvalMode}, checkoverflow::CheckOverflow, + correlation::Correlation, covariance::Covariance, if_expr::IfExpr, scalar_funcs::create_comet_physical_fun, @@ -1310,6 +1311,18 @@ impl PhysicalPlanner { ))), } } + AggExprStruct::Correlation(expr) => { + let child1 = self.create_expr(expr.child1.as_ref().unwrap(), schema.clone())?; + let child2 = self.create_expr(expr.child2.as_ref().unwrap(), schema.clone())?; + let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); + Ok(Arc::new(Correlation::new( + child1, + child2, + "correlation", + datatype, + expr.null_on_divide_by_zero, + ))) + } } } diff --git a/core/src/execution/proto/expr.proto b/core/src/execution/proto/expr.proto index ee3de865a..be85e8a92 100644 --- a/core/src/execution/proto/expr.proto +++ b/core/src/execution/proto/expr.proto @@ -96,6 +96,7 @@ message AggExpr { CovPopulation covPopulation = 13; Variance variance = 14; Stddev stddev = 15; + Correlation correlation = 16; } } @@ -186,6 +187,13 @@ message Stddev { StatisticsType stats_type = 4; } +message Correlation { + Expr child1 = 1; + Expr child2 = 2; + bool null_on_divide_by_zero = 3; + DataType datatype = 4; +} + message Literal { oneof value { bool bool_val = 1; diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index 38c86c727..521699d34 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -109,3 +109,4 @@ The following Spark expressions are currently available: - VarianceSamp - StddevPop - StddevSamp + - Corr 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 cf7c86a9f..6333650dd 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Average, BitAndAgg, BitOrAgg, BitXorAgg, Count, CovPopulation, CovSample, Final, First, Last, Max, Min, Partial, StddevPop, StddevSamp, Sum, VariancePop, VarianceSamp} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Average, BitAndAgg, BitOrAgg, BitXorAgg, Corr, Count, CovPopulation, CovSample, Final, First, Last, Max, Min, Partial, StddevPop, StddevSamp, Sum, VariancePop, VarianceSamp} import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke import org.apache.spark.sql.catalyst.optimizer.{BuildRight, NormalizeNaNAndZero} import org.apache.spark.sql.catalyst.plans._ @@ -547,6 +547,27 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim withInfo(aggExpr, child) None } + case corr @ Corr(child1, child2, nullOnDivideByZero) => + val child1Expr = exprToProto(child1, inputs, binding) + val child2Expr = exprToProto(child2, inputs, binding) + val dataType = serializeDataType(corr.dataType) + + if (child1Expr.isDefined && child2Expr.isDefined && dataType.isDefined) { + val corrBuilder = ExprOuterClass.Correlation.newBuilder() + corrBuilder.setChild1(child1Expr.get) + corrBuilder.setChild2(child2Expr.get) + corrBuilder.setNullOnDivideByZero(nullOnDivideByZero) + corrBuilder.setDatatype(dataType.get) + + Some( + ExprOuterClass.AggExpr + .newBuilder() + .setCorrelation(corrBuilder) + .build()) + } else { + withInfo(aggExpr, child1, child2) + None + } case fn => val msg = s"unsupported Spark aggregate function: ${fn.prettyName}" emitWarning(msg) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala index 310a24ee3..d36534ee8 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala @@ -1212,6 +1212,157 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + test("correlation") { + withSQLConf(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { + Seq(true, false).foreach { cometColumnShuffleEnabled => + withSQLConf( + CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> cometColumnShuffleEnabled.toString) { + Seq(true, false).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + Seq(true, false).foreach { nullOnDivideByZero => + withSQLConf( + "spark.sql.legacy.statisticalAggregate" -> nullOnDivideByZero.toString) { + val table = "test" + withTable(table) { + sql( + s"create table $table(col1 double, col2 double, col3 double) using parquet") + sql(s"insert into $table values(1, 4, 1), (2, 5, 1), (3, 6, 2)") + val expectedNumOfCometAggregates = 2 + + checkSparkAnswerAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table", + expectedNumOfCometAggregates) + + checkSparkAnswerAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table GROUP BY col3", + expectedNumOfCometAggregates) + } + + withTable(table) { + sql( + s"create table $table(col1 double, col2 double, col3 double) using parquet") + sql(s"insert into $table values(1, 4, 3), (2, -5, 3), (3, 6, 1)") + val expectedNumOfCometAggregates = 2 + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table", + expectedNumOfCometAggregates) + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table GROUP BY col3", + expectedNumOfCometAggregates) + } + + withTable(table) { + sql( + s"create table $table(col1 double, col2 double, col3 double) using parquet") + sql(s"insert into $table values(1.1, 4.1, 2.3), (2, 5, 1.5), (3, 6, 2.3)") + val expectedNumOfCometAggregates = 2 + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table", + expectedNumOfCometAggregates) + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table GROUP BY col3", + expectedNumOfCometAggregates) + } + + withTable(table) { + sql( + s"create table $table(col1 double, col2 double, col3 double) using parquet") + sql(s"insert into $table values(1, 4, 1), (2, 5, 2), (3, 6, 3), (1.1, 4.4, 1), (2.2, 5.5, 2), (3.3, 6.6, 3)") + val expectedNumOfCometAggregates = 2 + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table", + expectedNumOfCometAggregates) + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table GROUP BY col3", + expectedNumOfCometAggregates) + } + + withTable(table) { + sql(s"create table $table(col1 int, col2 int, col3 int) using parquet") + sql(s"insert into $table values(1, 4, 1), (2, 5, 2), (3, 6, 3)") + val expectedNumOfCometAggregates = 2 + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table", + expectedNumOfCometAggregates) + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table GROUP BY col3", + expectedNumOfCometAggregates) + } + + withTable(table) { + sql(s"create table $table(col1 int, col2 int, col3 int) using parquet") + sql( + s"insert into $table values(1, 4, 2), (null, null, 2), (3, 6, 1), (3, 3, 1)") + val expectedNumOfCometAggregates = 2 + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table", + expectedNumOfCometAggregates) + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table GROUP BY col3", + expectedNumOfCometAggregates) + } + + withTable(table) { + sql(s"create table $table(col1 int, col2 int, col3 int) using parquet") + sql(s"insert into $table values(1, 4, 1), (null, 5, 1), (2, 5, 2), (9, null, 2), (3, 6, 2)") + val expectedNumOfCometAggregates = 2 + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table", + expectedNumOfCometAggregates) + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table GROUP BY col3", + expectedNumOfCometAggregates) + } + + withTable(table) { + sql(s"create table $table(col1 int, col2 int, col3 int) using parquet") + sql(s"insert into $table values(null, null, 1), (1, 2, 1), (null, null, 2)") + val expectedNumOfCometAggregates = 2 + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table", + expectedNumOfCometAggregates) + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table GROUP BY col3", + expectedNumOfCometAggregates) + } + + withTable(table) { + sql(s"create table $table(col1 int, col2 int, col3 int) using parquet") + sql( + s"insert into $table values(null, null, 1), (null, null, 1), (null, null, 2)") + val expectedNumOfCometAggregates = 2 + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table", + expectedNumOfCometAggregates) + + checkSparkAnswerWithTolAndNumOfAggregates( + s"SELECT corr(col1, col2) FROM $table GROUP BY col3", + expectedNumOfCometAggregates) + } + } + } + } + } + } + } + } + } + protected def checkSparkAnswerAndNumOfAggregates(query: String, numAggregates: Int): Unit = { val df = sql(query) checkSparkAnswer(df) From a7272b90a7cc903c0b2c7482f1f44c2d325b407e Mon Sep 17 00:00:00 2001 From: vidyasankarv Date: Thu, 23 May 2024 22:58:54 +0530 Subject: [PATCH 07/14] feat: Implement Spark-compatible CAST from String to Date (#383) * support casting DateType in comet * Use NaiveDate methods for parsing dates and remove regex * remove macro for date parsing * compute correct days since epoch. * Run String to Date test without fuzzy test * port spark string to date processing logic for cast * put in fixes for clippy and scalafix issues. * handle non digit characters when parsing segement bytes. * add note on compatability * add negative tests for String to Date test * simplify byte digit check * propagate error correctly when a date cannot be parsed * add fuzz test with unsupported date filtering * add test for string array cast to date * use UNIX_EPOCH constant from NaiveDateTime * fix cargo clippy error - collapse else if * do not run string to date test on spark-3.2 * do not run string to date test on spark-3.2 * add failing fuzz test dates to tests and remove failing fuzz test * remove unused date pattern * add specific match for casting dictionary to date --- .../execution/datafusion/expressions/cast.rs | 379 +++++++++++++++++- docs/source/user-guide/compatibility.md | 1 + .../apache/comet/expressions/CometCast.scala | 2 +- .../org/apache/comet/CometCastSuite.scala | 68 +++- 4 files changed, 429 insertions(+), 21 deletions(-) diff --git a/core/src/execution/datafusion/expressions/cast.rs b/core/src/execution/datafusion/expressions/cast.rs index f68732fb1..fd1f9166d 100644 --- a/core/src/execution/datafusion/expressions/cast.rs +++ b/core/src/execution/datafusion/expressions/cast.rs @@ -22,7 +22,6 @@ use std::{ sync::Arc, }; -use crate::errors::{CometError, CometResult}; use arrow::{ compute::{cast_with_options, CastOptions}, datatypes::{ @@ -33,23 +32,27 @@ use arrow::{ util::display::FormatOptions, }; use arrow_array::{ - types::{Int16Type, Int32Type, Int64Type, Int8Type}, + types::{Date32Type, Int16Type, Int32Type, Int64Type, Int8Type}, Array, ArrayRef, BooleanArray, Decimal128Array, Float32Array, Float64Array, GenericStringArray, Int16Array, Int32Array, Int64Array, Int8Array, OffsetSizeTrait, PrimitiveArray, }; use arrow_schema::{DataType, Schema}; -use chrono::{TimeZone, Timelike}; +use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; use datafusion::logical_expr::ColumnarValue; use datafusion_common::{internal_err, Result as DataFusionResult, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use num::{cast::AsPrimitive, traits::CheckedNeg, CheckedSub, Integer, Num, ToPrimitive}; use regex::Regex; -use crate::execution::datafusion::expressions::utils::{ - array_with_timezone, down_cast_any_ref, spark_cast, +use crate::{ + errors::{CometError, CometResult}, + execution::datafusion::expressions::utils::{ + array_with_timezone, down_cast_any_ref, spark_cast, + }, }; static TIMESTAMP_FORMAT: Option<&str> = Some("%Y-%m-%d %H:%M:%S%.f"); + static CAST_OPTIONS: CastOptions = CastOptions { safe: true, format_options: FormatOptions::new() @@ -511,6 +514,31 @@ impl Cast { (DataType::Utf8, DataType::Timestamp(_, _)) => { Self::cast_string_to_timestamp(&array, to_type, self.eval_mode)? } + (DataType::Utf8, DataType::Date32) => { + Self::cast_string_to_date(&array, to_type, self.eval_mode)? + } + (DataType::Dictionary(key_type, value_type), DataType::Date32) + if key_type.as_ref() == &DataType::Int32 + && (value_type.as_ref() == &DataType::Utf8 + || value_type.as_ref() == &DataType::LargeUtf8) => + { + match value_type.as_ref() { + DataType::Utf8 => { + let unpacked_array = + cast_with_options(&array, &DataType::Utf8, &CAST_OPTIONS)?; + Self::cast_string_to_date(&unpacked_array, to_type, self.eval_mode)? + } + DataType::LargeUtf8 => { + let unpacked_array = + cast_with_options(&array, &DataType::LargeUtf8, &CAST_OPTIONS)?; + Self::cast_string_to_date(&unpacked_array, to_type, self.eval_mode)? + } + dt => unreachable!( + "{}", + format!("invalid value type {dt} for dictionary-encoded string array") + ), + } + } (DataType::Int64, DataType::Int32) | (DataType::Int64, DataType::Int16) | (DataType::Int64, DataType::Int8) @@ -635,6 +663,38 @@ impl Cast { Ok(cast_array) } + fn cast_string_to_date( + array: &ArrayRef, + to_type: &DataType, + eval_mode: EvalMode, + ) -> CometResult { + let string_array = array + .as_any() + .downcast_ref::>() + .expect("Expected a string array"); + + let cast_array: ArrayRef = match to_type { + DataType::Date32 => { + let len = string_array.len(); + let mut cast_array = PrimitiveArray::::builder(len); + for i in 0..len { + if !string_array.is_null(i) { + match date_parser(string_array.value(i), eval_mode) { + Ok(Some(cast_value)) => cast_array.append_value(cast_value), + Ok(None) => cast_array.append_null(), + Err(e) => return Err(e), + } + } else { + cast_array.append_null() + } + } + Arc::new(cast_array.finish()) as ArrayRef + } + _ => unreachable!("Invalid data type {:?} in cast from string", to_type), + }; + Ok(cast_array) + } + fn cast_string_to_timestamp( array: &ArrayRef, to_type: &DataType, @@ -858,7 +918,7 @@ impl Cast { i32, "FLOAT", "INT", - std::i32::MAX, + i32::MAX, "{:e}" ), (DataType::Float32, DataType::Int64) => cast_float_to_int32_up!( @@ -870,7 +930,7 @@ impl Cast { i64, "FLOAT", "BIGINT", - std::i64::MAX, + i64::MAX, "{:e}" ), (DataType::Float64, DataType::Int8) => cast_float_to_int16_down!( @@ -904,7 +964,7 @@ impl Cast { i32, "DOUBLE", "INT", - std::i32::MAX, + i32::MAX, "{:e}D" ), (DataType::Float64, DataType::Int64) => cast_float_to_int32_up!( @@ -916,7 +976,7 @@ impl Cast { i64, "DOUBLE", "BIGINT", - std::i64::MAX, + i64::MAX, "{:e}D" ), (DataType::Decimal128(precision, scale), DataType::Int8) => { @@ -936,7 +996,7 @@ impl Cast { Int32Array, i32, "INT", - std::i32::MAX, + i32::MAX, *precision, *scale ) @@ -948,7 +1008,7 @@ impl Cast { Int64Array, i64, "BIGINT", - std::i64::MAX, + i64::MAX, *precision, *scale ) @@ -1264,15 +1324,15 @@ fn timestamp_parser(value: &str, eval_mode: EvalMode) -> CometResult } if timestamp.is_none() { - if eval_mode == EvalMode::Ansi { - return Err(CometError::CastInvalidValue { + return if eval_mode == EvalMode::Ansi { + Err(CometError::CastInvalidValue { value: value.to_string(), from_type: "STRING".to_string(), to_type: "TIMESTAMP".to_string(), - }); + }) } else { - return Ok(None); - } + Ok(None) + }; } match timestamp { @@ -1409,13 +1469,136 @@ fn parse_str_to_time_only_timestamp(value: &str) -> CometResult> { Ok(Some(timestamp)) } +//a string to date parser - port of spark's SparkDateTimeUtils#stringToDate. +fn date_parser(date_str: &str, eval_mode: EvalMode) -> CometResult> { + // local functions + fn get_trimmed_start(bytes: &[u8]) -> usize { + let mut start = 0; + while start < bytes.len() && is_whitespace_or_iso_control(bytes[start]) { + start += 1; + } + start + } + + fn get_trimmed_end(start: usize, bytes: &[u8]) -> usize { + let mut end = bytes.len() - 1; + while end > start && is_whitespace_or_iso_control(bytes[end]) { + end -= 1; + } + end + 1 + } + + fn is_whitespace_or_iso_control(byte: u8) -> bool { + byte.is_ascii_whitespace() || byte.is_ascii_control() + } + + fn is_valid_digits(segment: i32, digits: usize) -> bool { + // An integer is able to represent a date within [+-]5 million years. + let max_digits_year = 7; + //year (segment 0) can be between 4 to 7 digits, + //month and day (segment 1 and 2) can be between 1 to 2 digits + (segment == 0 && digits >= 4 && digits <= max_digits_year) + || (segment != 0 && digits > 0 && digits <= 2) + } + + fn return_result(date_str: &str, eval_mode: EvalMode) -> CometResult> { + if eval_mode == EvalMode::Ansi { + Err(CometError::CastInvalidValue { + value: date_str.to_string(), + from_type: "STRING".to_string(), + to_type: "DATE".to_string(), + }) + } else { + Ok(None) + } + } + // end local functions + + if date_str.is_empty() { + return return_result(date_str, eval_mode); + } + + //values of date segments year, month and day defaulting to 1 + let mut date_segments = [1, 1, 1]; + let mut sign = 1; + let mut current_segment = 0; + let mut current_segment_value = 0; + let mut current_segment_digits = 0; + let bytes = date_str.as_bytes(); + + let mut j = get_trimmed_start(bytes); + let str_end_trimmed = get_trimmed_end(j, bytes); + + if j == str_end_trimmed { + return return_result(date_str, eval_mode); + } + + //assign a sign to the date + if bytes[j] == b'-' || bytes[j] == b'+' { + sign = if bytes[j] == b'-' { -1 } else { 1 }; + j += 1; + } + + //loop to the end of string until we have processed 3 segments, + //exit loop on encountering any space ' ' or 'T' after the 3rd segment + while j < str_end_trimmed && (current_segment < 3 && !(bytes[j] == b' ' || bytes[j] == b'T')) { + let b = bytes[j]; + if current_segment < 2 && b == b'-' { + //check for validity of year and month segments if current byte is separator + if !is_valid_digits(current_segment, current_segment_digits) { + return return_result(date_str, eval_mode); + } + //if valid update corresponding segment with the current segment value. + date_segments[current_segment as usize] = current_segment_value; + current_segment_value = 0; + current_segment_digits = 0; + current_segment += 1; + } else if !b.is_ascii_digit() { + return return_result(date_str, eval_mode); + } else { + //increment value of current segment by the next digit + let parsed_value = (b - b'0') as i32; + current_segment_value = current_segment_value * 10 + parsed_value; + current_segment_digits += 1; + } + j += 1; + } + + //check for validity of last segment + if !is_valid_digits(current_segment, current_segment_digits) { + return return_result(date_str, eval_mode); + } + + if current_segment < 2 && j < str_end_trimmed { + // For the `yyyy` and `yyyy-[m]m` formats, entire input must be consumed. + return return_result(date_str, eval_mode); + } + + date_segments[current_segment as usize] = current_segment_value; + + match NaiveDate::from_ymd_opt( + sign * date_segments[0], + date_segments[1] as u32, + date_segments[2] as u32, + ) { + Some(date) => { + let duration_since_epoch = date + .signed_duration_since(NaiveDateTime::UNIX_EPOCH.date()) + .num_days(); + Ok(Some(duration_since_epoch.to_i32().unwrap())) + } + None => Ok(None), + } +} + #[cfg(test)] mod tests { - use super::*; use arrow::datatypes::TimestampMicrosecondType; use arrow_array::StringArray; use arrow_schema::TimeUnit; + use super::*; + #[test] fn timestamp_parser_test() { // write for all formats @@ -1480,6 +1663,168 @@ mod tests { assert_eq!(result.len(), 2); } + #[test] + fn date_parser_test() { + for date in &[ + "2020", + "2020-01", + "2020-01-01", + "02020-01-01", + "002020-01-01", + "0002020-01-01", + "2020-1-1", + "2020-01-01 ", + "2020-01-01T", + ] { + for eval_mode in &[EvalMode::Legacy, EvalMode::Ansi, EvalMode::Try] { + assert_eq!(date_parser(*date, *eval_mode).unwrap(), Some(18262)); + } + } + + //dates in invalid formats + for date in &[ + "abc", + "", + "not_a_date", + "3/", + "3/12", + "3/12/2020", + "3/12/2002 T", + "202", + "2020-010-01", + "2020-10-010", + "2020-10-010T", + "--262143-12-31", + "--262143-12-31 ", + ] { + for eval_mode in &[EvalMode::Legacy, EvalMode::Try] { + assert_eq!(date_parser(*date, *eval_mode).unwrap(), None); + } + assert!(date_parser(*date, EvalMode::Ansi).is_err()); + } + + for date in &["-3638-5"] { + for eval_mode in &[EvalMode::Legacy, EvalMode::Try, EvalMode::Ansi] { + assert_eq!(date_parser(*date, *eval_mode).unwrap(), Some(-2048160)); + } + } + + //Naive Date only supports years 262142 AD to 262143 BC + //returns None for dates out of range supported by Naive Date. + for date in &[ + "-262144-1-1", + "262143-01-1", + "262143-1-1", + "262143-01-1 ", + "262143-01-01T ", + "262143-1-01T 1234", + "-0973250", + ] { + for eval_mode in &[EvalMode::Legacy, EvalMode::Try, EvalMode::Ansi] { + assert_eq!(date_parser(*date, *eval_mode).unwrap(), None); + } + } + } + + #[test] + fn test_cast_string_to_date() { + let array: ArrayRef = Arc::new(StringArray::from(vec![ + Some("2020"), + Some("2020-01"), + Some("2020-01-01"), + Some("2020-01-01T"), + ])); + + let result = + Cast::cast_string_to_date(&array, &DataType::Date32, EvalMode::Legacy).unwrap(); + + let date32_array = result + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(date32_array.len(), 4); + date32_array + .iter() + .for_each(|v| assert_eq!(v.unwrap(), 18262)); + } + + #[test] + fn test_cast_string_array_with_valid_dates() { + let array_with_invalid_date: ArrayRef = Arc::new(StringArray::from(vec![ + Some("-262143-12-31"), + Some("\n -262143-12-31 "), + Some("-262143-12-31T \t\n"), + Some("\n\t-262143-12-31T\r"), + Some("-262143-12-31T 123123123"), + Some("\r\n-262143-12-31T \r123123123"), + Some("\n -262143-12-31T \n\t"), + ])); + + for eval_mode in &[EvalMode::Legacy, EvalMode::Try, EvalMode::Ansi] { + let result = + Cast::cast_string_to_date(&array_with_invalid_date, &DataType::Date32, *eval_mode) + .unwrap(); + + let date32_array = result + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(result.len(), 7); + date32_array + .iter() + .for_each(|v| assert_eq!(v.unwrap(), -96464928)); + } + } + + #[test] + fn test_cast_string_array_with_invalid_dates() { + let array_with_invalid_date: ArrayRef = Arc::new(StringArray::from(vec![ + Some("2020"), + Some("2020-01"), + Some("2020-01-01"), + //4 invalid dates + Some("2020-010-01T"), + Some("202"), + Some(" 202 "), + Some("\n 2020-\r8 "), + Some("2020-01-01T"), + ])); + + for eval_mode in &[EvalMode::Legacy, EvalMode::Try] { + let result = + Cast::cast_string_to_date(&array_with_invalid_date, &DataType::Date32, *eval_mode) + .unwrap(); + + let date32_array = result + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!( + date32_array.iter().collect::>(), + vec![ + Some(18262), + Some(18262), + Some(18262), + None, + None, + None, + None, + Some(18262) + ] + ); + } + + let result = + Cast::cast_string_to_date(&array_with_invalid_date, &DataType::Date32, EvalMode::Ansi); + match result { + Err(e) => assert!( + e.to_string().contains( + "[CAST_INVALID_INPUT] The value '2020-010-01T' of the type \"STRING\" cannot be cast to \"DATE\" because it is malformed") + ), + _ => panic!("Expected error"), + } + } + #[test] fn test_cast_string_as_i8() { // basic diff --git a/docs/source/user-guide/compatibility.md b/docs/source/user-guide/compatibility.md index a4ed9289f..a16fd1b21 100644 --- a/docs/source/user-guide/compatibility.md +++ b/docs/source/user-guide/compatibility.md @@ -115,6 +115,7 @@ The following cast operations are generally compatible with Spark except for the | string | integer | | | string | long | | | string | binary | | +| string | date | Only supports years between 262143 BC and 262142 AD | | date | string | | | timestamp | long | | | timestamp | decimal | | diff --git a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala index 795bdb428..11c5a53cc 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -119,7 +119,7 @@ object CometCast { Unsupported case DataTypes.DateType => // https://github.com/apache/datafusion-comet/issues/327 - Unsupported + Compatible(Some("Only supports years between 262143 BC and 262142 AD")) case DataTypes.TimestampType if timeZoneId.exists(tz => tz != "UTC") => Incompatible(Some(s"Cast will use UTC instead of $timeZoneId")) case DataTypes.TimestampType if evalMode == "ANSI" => diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 8caba14c6..1710090e2 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -22,6 +22,7 @@ package org.apache.comet import java.io.File import scala.util.Random +import scala.util.matching.Regex import org.apache.spark.sql.{CometTestBase, DataFrame, SaveMode} import org.apache.spark.sql.catalyst.expressions.Cast @@ -33,6 +34,7 @@ import org.apache.spark.sql.types.{DataType, DataTypes, DecimalType} import org.apache.comet.expressions.{CometCast, Compatible} class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { + import testImplicits._ /** Create a data generator using a fixed seed so that tests are reproducible */ @@ -53,6 +55,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { private val numericPattern = "0123456789deEf+-." + whitespaceChars private val datePattern = "0123456789/" + whitespaceChars + private val timestampPattern = "0123456789/:T" + whitespaceChars test("all valid cast combinations covered") { @@ -567,9 +570,68 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(gen.generateStrings(dataSize, numericPattern, 8).toDF("a"), DataTypes.BinaryType) } - ignore("cast StringType to DateType") { - // https://github.com/apache/datafusion-comet/issues/327 - castTest(gen.generateStrings(dataSize, datePattern, 8).toDF("a"), DataTypes.DateType) + test("cast StringType to DateType") { + // error message for invalid dates in Spark 3.2 not supported by Comet see below issue. + // https://github.com/apache/datafusion-comet/issues/440 + assume(CometSparkSessionExtensions.isSpark33Plus) + val validDates = Seq( + "262142-01-01", + "262142-01-01 ", + "262142-01-01T ", + "262142-01-01T 123123123", + "-262143-12-31", + "-262143-12-31 ", + "-262143-12-31T", + "-262143-12-31T ", + "-262143-12-31T 123123123", + "2020", + "2020-1", + "2020-1-1", + "2020-01", + "2020-01-01", + "2020-1-01 ", + "2020-01-1", + "02020-01-01", + "2020-01-01T", + "2020-10-01T 1221213", + "002020-01-01 ", + "0002020-01-01 123344", + "-3638-5") + val invalidDates = Seq( + "0", + "202", + "3/", + "3/3/", + "3/3/2020", + "3#3#2020", + "2020-010-01", + "2020-10-010", + "2020-10-010T", + "--262143-12-31", + "--262143-12-31T 1234 ", + "abc-def-ghi", + "abc-def-ghi jkl", + "2020-mar-20", + "not_a_date", + "T2", + "\t\n3938\n8", + "8701\t", + "\n8757", + "7593\t\t\t", + "\t9374 \n ", + "\n 9850 \t", + "\r\n\t9840", + "\t9629\n", + "\r\n 9629 \r\n", + "\r\n 962 \r\n", + "\r\n 62 \r\n") + + // due to limitations of NaiveDate we only support years between 262143 BC and 262142 AD" + val unsupportedYearPattern: Regex = "^\\s*[0-9]{5,}".r + val fuzzDates = gen + .generateStrings(dataSize, datePattern, 8) + .filterNot(str => unsupportedYearPattern.findFirstMatchIn(str).isDefined) + castTest((validDates ++ invalidDates ++ fuzzDates).toDF("a"), DataTypes.DateType) } test("cast StringType to TimestampType disabled by default") { From 507e475bd06fbb9cc738d61473a8341caa6daf74 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 23 May 2024 14:07:43 -0700 Subject: [PATCH 08/14] feat: Add COMET_SHUFFLE_MODE config to control Comet shuffle mode (#460) --- .../scala/org/apache/comet/CometConf.scala | 20 +++-- docs/source/user-guide/configs.md | 2 +- docs/source/user-guide/tuning.md | 22 +++--- .../comet/CometSparkSessionExtensions.scala | 36 ++++++--- .../apache/comet/CometExpressionSuite.scala | 4 +- .../comet/exec/CometAggregateSuite.scala | 75 +++++++++---------- .../exec/CometColumnarShuffleSuite.scala | 10 +-- .../apache/comet/exec/CometExecSuite.scala | 23 +++--- .../comet/exec/CometNativeShuffleSuite.scala | 2 +- .../spark/sql/CometTPCHQuerySuite.scala | 2 +- .../sql/benchmark/CometExecBenchmark.scala | 2 +- .../sql/benchmark/CometShuffleBenchmark.scala | 19 ++--- .../apache/comet/exec/CometExec3_4Suite.scala | 4 +- 13 files changed, 119 insertions(+), 102 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 463de90c2..5aee02f11 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -19,6 +19,7 @@ package org.apache.comet +import java.util.Locale import java.util.concurrent.TimeUnit import scala.collection.mutable.ListBuffer @@ -131,14 +132,17 @@ object CometConf { .booleanConf .createWithDefault(false) - val COMET_COLUMNAR_SHUFFLE_ENABLED: ConfigEntry[Boolean] = - conf("spark.comet.columnar.shuffle.enabled") - .doc( - "Whether to enable Arrow-based columnar shuffle for Comet and Spark regular operators. " + - "If this is enabled, Comet prefers columnar shuffle than native shuffle. " + - "By default, this config is true.") - .booleanConf - .createWithDefault(true) + val COMET_SHUFFLE_MODE: ConfigEntry[String] = conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.mode") + .doc("The mode of Comet shuffle. This config is only effective if Comet shuffle " + + "is enabled. Available modes are 'native', 'jvm', and 'auto'. " + + "'native' is for native shuffle which has best performance in general. " + + "'jvm' is for jvm-based columnar shuffle which has higher coverage than native shuffle. " + + "'auto' is for Comet to choose the best shuffle mode based on the query plan. " + + "By default, this config is 'jvm'.") + .stringConf + .transform(_.toLowerCase(Locale.ROOT)) + .checkValues(Set("native", "jvm", "auto")) + .createWithDefault("jvm") val COMET_SHUFFLE_ENFORCE_MODE_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.shuffle.enforceMode.enabled") diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 0204b0c54..eb349b349 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -29,7 +29,6 @@ Comet provides the following configuration settings. | spark.comet.columnar.shuffle.async.enabled | Whether to enable asynchronous shuffle for Arrow-based shuffle. By default, this config is false. | false | | spark.comet.columnar.shuffle.async.max.thread.num | Maximum number of threads on an executor used for Comet async columnar shuffle. By default, this config is 100. This is the upper bound of total number of shuffle threads per executor. In other words, if the number of cores * the number of shuffle threads per task `spark.comet.columnar.shuffle.async.thread.num` is larger than this config. Comet will use this config as the number of shuffle threads per executor instead. | 100 | | spark.comet.columnar.shuffle.async.thread.num | Number of threads used for Comet async columnar shuffle per shuffle task. By default, this config is 3. Note that more threads means more memory requirement to buffer shuffle data before flushing to disk. Also, more threads may not always improve performance, and should be set based on the number of cores available. | 3 | -| spark.comet.columnar.shuffle.enabled | Whether to enable Arrow-based columnar shuffle for Comet and Spark regular operators. If this is enabled, Comet prefers columnar shuffle than native shuffle. By default, this config is true. | true | | spark.comet.columnar.shuffle.memory.factor | Fraction of Comet memory to be allocated per executor process for Comet shuffle. Comet memory size is specified by `spark.comet.memoryOverhead` or calculated by `spark.comet.memory.overhead.factor` * `spark.executor.memory`. By default, this config is 1.0. | 1.0 | | spark.comet.debug.enabled | Whether to enable debug mode for Comet. By default, this config is false. When enabled, Comet will do additional checks for debugging purpose. For example, validating array when importing arrays from JVM at native side. Note that these checks may be expensive in performance and should only be enabled for debugging purpose. | false | | spark.comet.enabled | Whether to enable Comet extension for Spark. When this is turned on, Spark will use Comet to read Parquet data source. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. By default, this config is the value of the env var `ENABLE_COMET` if set, or true otherwise. | true | @@ -39,6 +38,7 @@ Comet provides the following configuration settings. | spark.comet.exec.memoryFraction | The fraction of memory from Comet memory overhead that the native memory manager can use for execution. The purpose of this config is to set aside memory for untracked data structures, as well as imprecise size estimation during memory acquisition. Default value is 0.7. | 0.7 | | spark.comet.exec.shuffle.codec | The codec of Comet native shuffle used to compress shuffle data. Only zstd is supported. | zstd | | spark.comet.exec.shuffle.enabled | Whether to enable Comet native shuffle. By default, this config is false. Note that this requires setting 'spark.shuffle.manager' to 'org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager'. 'spark.shuffle.manager' must be set before starting the Spark application and cannot be changed during the application. | false | +| spark.comet.exec.shuffle.mode | The mode of Comet shuffle. This config is only effective if Comet shuffle is enabled. Available modes are 'native', 'jvm', and 'auto'. 'native' is for native shuffle which has best performance in general. 'jvm' is for jvm-based columnar shuffle which has higher coverage than native shuffle. 'auto' is for Comet to choose the best shuffle mode based on the query plan. By default, this config is 'jvm'. | jvm | | spark.comet.explainFallback.enabled | When this setting is enabled, Comet will provide logging explaining the reason(s) why a query stage cannot be executed natively. | false | | spark.comet.memory.overhead.factor | Fraction of executor memory to be allocated as additional non-heap memory per executor process for Comet. Default value is 0.2. | 0.2 | | spark.comet.memory.overhead.min | Minimum amount of additional memory to be allocated per executor process for Comet, in MiB. | 402653184b | diff --git a/docs/source/user-guide/tuning.md b/docs/source/user-guide/tuning.md index 01fa7bdbe..5a3100bd0 100644 --- a/docs/source/user-guide/tuning.md +++ b/docs/source/user-guide/tuning.md @@ -39,22 +39,26 @@ It must be set before the Spark context is created. You can enable or disable Co at runtime by setting `spark.comet.exec.shuffle.enabled` to `true` or `false`. Once it is disabled, Comet will fallback to the default Spark shuffle manager. -### Columnar Shuffle +### Shuffle Mode -By default, once `spark.comet.exec.shuffle.enabled` is enabled, Comet uses columnar shuffle +Comet provides three shuffle modes: Columnar Shuffle, Native Shuffle and Auto Mode. + +#### Columnar Shuffle + +By default, once `spark.comet.exec.shuffle.enabled` is enabled, Comet uses JVM-based columnar shuffle to improve the performance of shuffle operations. Columnar shuffle supports HashPartitioning, -RoundRobinPartitioning, RangePartitioning and SinglePartitioning. +RoundRobinPartitioning, RangePartitioning and SinglePartitioning. This mode has the highest +query coverage. -Columnar shuffle can be disabled by setting `spark.comet.columnar.shuffle.enabled` to `false`. +Columnar shuffle can be enabled by setting `spark.comet.exec.shuffle.mode` to `jvm`. -### Native Shuffle +#### Native Shuffle Comet also provides a fully native shuffle implementation that can be used to improve the performance. -To enable native shuffle, just disable `spark.comet.columnar.shuffle.enabled`. +To enable native shuffle, just set `spark.comet.exec.shuffle.mode` to `native` Native shuffle only supports HashPartitioning and SinglePartitioning. +### Auto Mode - - - +`spark.comet.exec.shuffle.mode` to `auto` will let Comet choose the best shuffle mode based on the query plan. diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 85a19f55c..168d2bb52 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.comet.CometConf._ -import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometColumnarShuffleEnabled, isCometEnabled, isCometExecEnabled, isCometOperatorEnabled, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSchemaSupported, isSpark34Plus, shouldApplyRowToColumnar, withInfo, withInfos} +import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometOperatorEnabled, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSchemaSupported, isSpark34Plus, shouldApplyRowToColumnar, withInfo, withInfos} import org.apache.comet.parquet.{CometParquetScan, SupportsComet} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde @@ -197,7 +197,7 @@ class CometSparkSessionExtensions private def applyCometShuffle(plan: SparkPlan): SparkPlan = { plan.transformUp { case s: ShuffleExchangeExec - if isCometPlan(s.child) && !isCometColumnarShuffleEnabled(conf) && + if isCometPlan(s.child) && isCometNativeShuffleMode(conf) && QueryPlanSerde.supportPartitioning(s.child.output, s.outputPartitioning)._1 => logInfo("Comet extension enabled for Native Shuffle") @@ -209,8 +209,8 @@ class CometSparkSessionExtensions // Columnar shuffle for regular Spark operators (not Comet) and Comet operators // (if configured) case s: ShuffleExchangeExec - if (!s.child.supportsColumnar || isCometPlan( - s.child)) && isCometColumnarShuffleEnabled(conf) && + if (!s.child.supportsColumnar || isCometPlan(s.child)) && isCometJVMShuffleMode( + conf) && QueryPlanSerde.supportPartitioningTypes(s.child.output)._1 && !isShuffleOperator(s.child) => logInfo("Comet extension enabled for JVM Columnar Shuffle") @@ -641,7 +641,7 @@ class CometSparkSessionExtensions // Native shuffle for Comet operators case s: ShuffleExchangeExec if isCometShuffleEnabled(conf) && - !isCometColumnarShuffleEnabled(conf) && + isCometNativeShuffleMode(conf) && QueryPlanSerde.supportPartitioning(s.child.output, s.outputPartitioning)._1 => logInfo("Comet extension enabled for Native Shuffle") @@ -662,7 +662,7 @@ class CometSparkSessionExtensions // If the child of ShuffleExchangeExec is also a ShuffleExchangeExec, we should not // convert it to CometColumnarShuffle, case s: ShuffleExchangeExec - if isCometShuffleEnabled(conf) && isCometColumnarShuffleEnabled(conf) && + if isCometShuffleEnabled(conf) && isCometJVMShuffleMode(conf) && QueryPlanSerde.supportPartitioningTypes(s.child.output)._1 && !isShuffleOperator(s.child) => logInfo("Comet extension enabled for JVM Columnar Shuffle") @@ -684,19 +684,19 @@ class CometSparkSessionExtensions case s: ShuffleExchangeExec => val isShuffleEnabled = isCometShuffleEnabled(conf) val reason = getCometShuffleNotEnabledReason(conf).getOrElse("no reason available") - val msg1 = createMessage(!isShuffleEnabled, s"Native shuffle is not enabled: $reason") - val columnarShuffleEnabled = isCometColumnarShuffleEnabled(conf) + val msg1 = createMessage(!isShuffleEnabled, s"Comet shuffle is not enabled: $reason") + val columnarShuffleEnabled = isCometJVMShuffleMode(conf) val msg2 = createMessage( isShuffleEnabled && !columnarShuffleEnabled && !QueryPlanSerde .supportPartitioning(s.child.output, s.outputPartitioning) ._1, - "Shuffle: " + + "Native shuffle: " + s"${QueryPlanSerde.supportPartitioning(s.child.output, s.outputPartitioning)._2}") val msg3 = createMessage( isShuffleEnabled && columnarShuffleEnabled && !QueryPlanSerde .supportPartitioningTypes(s.child.output) ._1, - s"Columnar shuffle: ${QueryPlanSerde.supportPartitioningTypes(s.child.output)._2}") + s"JVM shuffle: ${QueryPlanSerde.supportPartitioningTypes(s.child.output)._2}") withInfo(s, Seq(msg1, msg2, msg3).flatten.mkString(",")) s @@ -966,8 +966,20 @@ object CometSparkSessionExtensions extends Logging { COMET_EXEC_ENABLED.get(conf) } - private[comet] def isCometColumnarShuffleEnabled(conf: SQLConf): Boolean = { - COMET_COLUMNAR_SHUFFLE_ENABLED.get(conf) + private[comet] def isCometNativeShuffleMode(conf: SQLConf): Boolean = { + COMET_SHUFFLE_MODE.get(conf) match { + case "native" => true + case "auto" => true + case _ => false + } + } + + private[comet] def isCometJVMShuffleMode(conf: SQLConf): Boolean = { + COMET_SHUFFLE_MODE.get(conf) match { + case "jvm" => true + case "auto" => true + case _ => false + } } private[comet] def isCometAllOperatorEnabled(conf: SQLConf): Boolean = { diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 98a2bad02..6ca4baf60 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -1421,7 +1421,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { ( s"SELECT sum(c0), sum(c2) from $table group by c1", Set( - "Native shuffle is not enabled: spark.comet.exec.shuffle.enabled is not enabled", + "Comet shuffle is not enabled: spark.comet.exec.shuffle.enabled is not enabled", "AQEShuffleRead is not supported")), ( "SELECT A.c1, A.sum_c0, A.sum_c2, B.casted from " @@ -1429,7 +1429,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { + s"(SELECT c1, cast(make_interval(c0, c1, c0, c1, c0, c0, c2) as string) as casted from $table) as B " + "where A.c1 = B.c1 ", Set( - "Native shuffle is not enabled: spark.comet.exec.shuffle.enabled is not enabled", + "Comet shuffle is not enabled: spark.comet.exec.shuffle.enabled is not enabled", "AQEShuffleRead is not supported", "make_interval is not supported", "BroadcastExchange is not supported", diff --git a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala index d36534ee8..ca7bc7df0 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala @@ -44,7 +44,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val df1 = sql("SELECT count(DISTINCT 2), count(DISTINCT 2,3)") checkSparkAnswer(df1) @@ -57,7 +57,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { checkSparkAnswer(sql(""" |SELECT | lag(123, 100, 321) OVER (ORDER BY id) as lag, @@ -78,7 +78,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val df1 = Seq( ("a", "b", "c"), ("a", "b", "c"), @@ -99,7 +99,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val df = sql("SELECT LAST(n) FROM lowerCaseData") checkSparkAnswer(df) } @@ -114,7 +114,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val df = sql("select sum(a), avg(a) from allNulls") checkSparkAnswer(df) } @@ -125,7 +125,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { withTempDir { dir => val path = new Path(dir.toURI.toString, "test") makeParquetFile(path, 10000, 10, false) @@ -141,7 +141,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { Seq(true, false).foreach { dictionaryEnabled => withTempDir { dir => val path = new Path(dir.toURI.toString, "test") @@ -160,7 +160,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { sql( "CREATE TABLE lineitem(l_extendedprice DOUBLE, l_quantity DOUBLE, l_partkey STRING) USING PARQUET") @@ -197,7 +197,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> EliminateSorts.ruleName, CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { Seq(true, false).foreach { dictionaryEnabled => withTempDir { dir => val path = new Path(dir.toURI.toString, "test") @@ -216,7 +216,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "false", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "false") { + CometConf.COMET_SHUFFLE_MODE.key -> "native") { withTable(table) { sql(s"CREATE TABLE $table(col DECIMAL(5, 2)) USING PARQUET") sql(s"INSERT INTO TABLE $table VALUES (CAST(12345.01 AS DECIMAL(5, 2)))") @@ -316,7 +316,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { Seq(true, false).foreach { dictionaryEnabled => withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> nativeShuffleEnabled.toString, - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "false") { + CometConf.COMET_SHUFFLE_MODE.key -> "native") { withParquetTable( (0 until 100).map(i => (i, (i % 10).toString)), "tbl", @@ -497,7 +497,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { Seq(true, false).foreach { nativeShuffleEnabled => withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> nativeShuffleEnabled.toString, - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "false") { + CometConf.COMET_SHUFFLE_MODE.key -> "native") { withTempDir { dir => val path = new Path(dir.toURI.toString, "test") makeParquetFile(path, 1000, 20, dictionaryEnabled) @@ -686,7 +686,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("test final count") { withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "false") { + CometConf.COMET_SHUFFLE_MODE.key -> "native") { Seq(false, true).foreach { dictionaryEnabled => withParquetTable((0 until 5).map(i => (i, i % 2)), "tbl", dictionaryEnabled) { checkSparkAnswerAndNumOfAggregates("SELECT _2, COUNT(_1) FROM tbl GROUP BY _2", 2) @@ -703,7 +703,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("test final min/max") { withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "false") { + CometConf.COMET_SHUFFLE_MODE.key -> "native") { Seq(true, false).foreach { dictionaryEnabled => withParquetTable((0 until 5).map(i => (i, i % 2)), "tbl", dictionaryEnabled) { checkSparkAnswerAndNumOfAggregates( @@ -724,7 +724,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("test final min/max/count with result expressions") { withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "false") { + CometConf.COMET_SHUFFLE_MODE.key -> "native") { Seq(true, false).foreach { dictionaryEnabled => withParquetTable((0 until 5).map(i => (i, i % 2)), "tbl", dictionaryEnabled) { checkSparkAnswerAndNumOfAggregates( @@ -759,7 +759,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("test final sum") { withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "false") { + CometConf.COMET_SHUFFLE_MODE.key -> "native") { Seq(false, true).foreach { dictionaryEnabled => withParquetTable((0L until 5L).map(i => (i, i % 2)), "tbl", dictionaryEnabled) { checkSparkAnswerAndNumOfAggregates( @@ -780,7 +780,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("test final avg") { withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "false") { + CometConf.COMET_SHUFFLE_MODE.key -> "native") { Seq(true, false).foreach { dictionaryEnabled => withParquetTable( (0 until 5).map(i => (i.toDouble, i.toDouble % 2)), @@ -805,7 +805,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "false") { + CometConf.COMET_SHUFFLE_MODE.key -> "native") { Seq(true, false).foreach { dictionaryEnabled => withSQLConf("parquet.enable.dictionary" -> dictionaryEnabled.toString) { val table = "t1" @@ -850,7 +850,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("avg null handling") { withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "false") { + CometConf.COMET_SHUFFLE_MODE.key -> "native") { val table = "t1" withTable(table) { sql(s"create table $table(a double, b double) using parquet") @@ -872,7 +872,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { Seq(true, false).foreach { nativeShuffleEnabled => withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> nativeShuffleEnabled.toString, - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "false", + CometConf.COMET_SHUFFLE_MODE.key -> "native", CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { withTempDir { dir => val path = new Path(dir.toURI.toString, "test") @@ -912,11 +912,11 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("distinct") { withSQLConf(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { - Seq(true, false).foreach { cometColumnShuffleEnabled => - withSQLConf( - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> cometColumnShuffleEnabled.toString) { + Seq("native", "jvm").foreach { cometShuffleMode => + withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> cometShuffleMode) { Seq(true, false).foreach { dictionary => withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val cometColumnShuffleEnabled = cometShuffleMode == "jvm" val table = "test" withTable(table) { sql(s"create table $table(col1 int, col2 int, col3 int) using parquet") @@ -970,7 +970,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_ENFORCE_MODE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { Seq(true, false).foreach { dictionary => withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { val table = "test" @@ -1016,9 +1016,8 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("test bool_and/bool_or") { withSQLConf(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { - Seq(true, false).foreach { cometColumnShuffleEnabled => - withSQLConf( - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> cometColumnShuffleEnabled.toString) { + Seq("native", "jvm").foreach { cometShuffleMode => + withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> cometShuffleMode) { Seq(true, false).foreach { dictionary => withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { val table = "test" @@ -1043,7 +1042,7 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("bitwise aggregate") { withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { Seq(true, false).foreach { dictionary => withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { val table = "test" @@ -1092,9 +1091,8 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("covar_pop and covar_samp") { withSQLConf(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { - Seq(true, false).foreach { cometColumnShuffleEnabled => - withSQLConf( - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> cometColumnShuffleEnabled.toString) { + Seq("native", "jvm").foreach { cometShuffleMode => + withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> cometShuffleMode) { Seq(true, false).foreach { dictionary => withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { val table = "test" @@ -1131,9 +1129,8 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("var_pop and var_samp") { withSQLConf(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { - Seq(true, false).foreach { cometColumnShuffleEnabled => - withSQLConf( - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> cometColumnShuffleEnabled.toString) { + Seq("native", "jvm").foreach { cometShuffleMode => + withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> cometShuffleMode) { Seq(true, false).foreach { dictionary => withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { Seq(true, false).foreach { nullOnDivideByZero => @@ -1171,9 +1168,8 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("stddev_pop and stddev_samp") { withSQLConf(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { - Seq(true, false).foreach { cometColumnShuffleEnabled => - withSQLConf( - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> cometColumnShuffleEnabled.toString) { + Seq("native", "jvm").foreach { cometShuffleMode => + withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> cometShuffleMode) { Seq(true, false).foreach { dictionary => withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { Seq(true, false).foreach { nullOnDivideByZero => @@ -1214,9 +1210,8 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("correlation") { withSQLConf(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { - Seq(true, false).foreach { cometColumnShuffleEnabled => - withSQLConf( - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> cometColumnShuffleEnabled.toString) { + Seq("jvm", "native").foreach { cometShuffleMode => + withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> cometShuffleMode) { Seq(true, false).foreach { dictionary => withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { Seq(true, false).foreach { nullOnDivideByZero => diff --git a/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala index 600f9c44f..c38be7c4a 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometColumnarShuffleSuite.scala @@ -54,7 +54,7 @@ abstract class CometColumnarShuffleSuite extends CometTestBase with AdaptiveSpar CometConf.COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED.key -> asyncShuffleEnable.toString, CometConf.COMET_COLUMNAR_SHUFFLE_SPILL_THRESHOLD.key -> numElementsForceSpillThreshold.toString, CometConf.COMET_EXEC_ENABLED.key -> "false", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_COLUMNAR_SHUFFLE_MEMORY_SIZE.key -> "1536m") { testFun @@ -963,7 +963,7 @@ class CometShuffleSuite extends CometColumnarShuffleSuite { SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { withParquetTable((0 until 10).map(i => (i, i % 5)), "tbl_a") { val df = sql("SELECT * FROM tbl_a") val shuffled = df @@ -983,7 +983,7 @@ class CometShuffleSuite extends CometColumnarShuffleSuite { SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { withParquetTable((0 until 10).map(i => (i, i % 5)), "tbl_a") { withParquetTable((0 until 10).map(i => (i % 10, i + 2)), "tbl_b") { val df = sql("SELECT * FROM tbl_a") @@ -1016,7 +1016,7 @@ class DisableAQECometShuffleSuite extends CometColumnarShuffleSuite { SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { withParquetTable((0 until 10).map(i => (i, i % 5)), "tbl_a") { withParquetTable((0 until 10).map(i => (i % 10, i + 2)), "tbl_b") { val df = sql("SELECT * FROM tbl_a") @@ -1061,7 +1061,7 @@ class CometShuffleEncryptionSuite extends CometTestBase { withSQLConf( CometConf.COMET_EXEC_ENABLED.key -> "false", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm", CometConf.COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED.key -> asyncEnabled.toString) { readParquetFile(path.toString) { df => val shuffled = df diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index c5fef022c..7c19890d3 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -134,14 +134,15 @@ class CometExecSuite extends CometTestBase { .toDF("c1", "c2") .createOrReplaceTempView("v") - Seq(true, false).foreach { columnarShuffle => + Seq("native", "jvm").foreach { columnarShuffleMode => withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> columnarShuffle.toString) { + CometConf.COMET_SHUFFLE_MODE.key -> columnarShuffleMode) { val df = sql("SELECT * FROM v where c1 = 1 order by c1, c2") val shuffle = find(df.queryExecution.executedPlan) { - case _: CometShuffleExchangeExec if columnarShuffle => true - case _: ShuffleExchangeExec if !columnarShuffle => true + case _: CometShuffleExchangeExec if columnarShuffleMode.equalsIgnoreCase("jvm") => + true + case _: ShuffleExchangeExec if !columnarShuffleMode.equalsIgnoreCase("jvm") => true case _ => false }.get assert(shuffle.logicalLink.isEmpty) @@ -179,7 +180,7 @@ class CometExecSuite extends CometTestBase { SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled, // `REQUIRE_ALL_CLUSTER_KEYS_FOR_DISTRIBUTION` is a new config in Spark 3.3+. "spark.sql.requireAllClusterKeysForDistribution" -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val df = Seq(("a", 1, 1), ("a", 2, 2), ("b", 1, 3), ("b", 1, 4)).toDF("key1", "key2", "value") val windowSpec = Window.partitionBy("key1", "key2").orderBy("value") @@ -318,7 +319,7 @@ class CometExecSuite extends CometTestBase { dataTypes.map { subqueryType => withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm", CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl") { var column1 = s"CAST(max(_1) AS $subqueryType)" @@ -499,7 +500,7 @@ class CometExecSuite extends CometTestBase { SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { withTable(tableName, dim) { sql( @@ -716,7 +717,7 @@ class CometExecSuite extends CometTestBase { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl") { val df = sql("SELECT * FROM tbl").sort($"_1".desc) checkSparkAnswerAndOperator(df) @@ -764,10 +765,10 @@ class CometExecSuite extends CometTestBase { } test("limit") { - Seq("true", "false").foreach { columnarShuffle => + Seq("native", "jvm").foreach { columnarShuffleMode => withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> columnarShuffle) { + CometConf.COMET_SHUFFLE_MODE.key -> columnarShuffleMode) { withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl_a") { val df = sql("SELECT * FROM tbl_a") .repartition(10, $"_1") @@ -1411,7 +1412,7 @@ class CometExecSuite extends CometTestBase { Seq("true", "false").foreach(aqe => { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqe, - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm", SQLConf.CACHE_VECTORIZED_READER_ENABLED.key -> "false") { spark .range(1000) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index d48ba1839..d17e4abf4 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -37,7 +37,7 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper super.test(testName, testTags: _*) { withSQLConf( CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "false", + CometConf.COMET_SHUFFLE_MODE.key -> "native", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { testFun } diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCHQuerySuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCHQuerySuite.scala index e8aac2619..1abe5faeb 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTPCHQuerySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCHQuerySuite.scala @@ -90,7 +90,7 @@ class CometTPCHQuerySuite extends QueryTest with CometTPCBase with SQLQueryTestH conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key, "true") - conf.set(CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key, "true") + conf.set(CometConf.COMET_SHUFFLE_MODE.key, "jvm") conf.set(MEMORY_OFFHEAP_ENABLED.key, "true") conf.set(MEMORY_OFFHEAP_SIZE.key, "2g") } diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometExecBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometExecBenchmark.scala index d6020ac69..bf4bfdbee 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometExecBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometExecBenchmark.scala @@ -131,7 +131,7 @@ object CometExecBenchmark extends CometBenchmarkBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { spark.sql( "SELECT (SELECT max(col1) AS parquetV1Table FROM parquetV1Table) AS a, " + "col2, col3 FROM parquetV1Table") diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometShuffleBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometShuffleBenchmark.scala index 865572811..30a2823cf 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometShuffleBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometShuffleBenchmark.scala @@ -106,7 +106,7 @@ object CometShuffleBenchmark extends CometBenchmarkBase { CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_PREFER_DICTIONARY_RATIO.key -> "1.0", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm", CometConf.COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED.key -> "false") { spark .sql( @@ -165,7 +165,7 @@ object CometShuffleBenchmark extends CometBenchmarkBase { CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_PREFER_DICTIONARY_RATIO.key -> "1.0", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm", CometConf.COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED.key -> "false") { spark .sql( @@ -222,7 +222,7 @@ object CometShuffleBenchmark extends CometBenchmarkBase { CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_PREFER_DICTIONARY_RATIO.key -> "1.0", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm", CometConf.COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED.key -> "false") { spark .sql("select c1 from parquetV1Table") @@ -238,7 +238,7 @@ object CometShuffleBenchmark extends CometBenchmarkBase { CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_PREFER_DICTIONARY_RATIO.key -> "2.0", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm", CometConf.COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED.key -> "false") { spark .sql("select c1 from parquetV1Table") @@ -254,7 +254,7 @@ object CometShuffleBenchmark extends CometBenchmarkBase { CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", CometConf.COMET_SHUFFLE_PREFER_DICTIONARY_RATIO.key -> "1000000000.0", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm", CometConf.COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED.key -> "false") { spark .sql("select c1 from parquetV1Table") @@ -321,7 +321,7 @@ object CometShuffleBenchmark extends CometBenchmarkBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm", CometConf.COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED.key -> "false") { spark .sql("select c1 from parquetV1Table") @@ -336,7 +336,7 @@ object CometShuffleBenchmark extends CometBenchmarkBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm", CometConf.COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED.key -> "true") { spark .sql("select c1 from parquetV1Table") @@ -409,7 +409,7 @@ object CometShuffleBenchmark extends CometBenchmarkBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", - CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { spark .sql(s"select $columns from parquetV1Table") .repartition(partitionNum, Column("c1")) @@ -422,7 +422,8 @@ object CometShuffleBenchmark extends CometBenchmarkBase { CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", - CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "native") { spark .sql(s"select $columns from parquetV1Table") .repartition(partitionNum, Column("c1")) diff --git a/spark/src/test/spark-3.4/org/apache/comet/exec/CometExec3_4Suite.scala b/spark/src/test/spark-3.4/org/apache/comet/exec/CometExec3_4Suite.scala index 32b76d9b0..019b4f030 100644 --- a/spark/src/test/spark-3.4/org/apache/comet/exec/CometExec3_4Suite.scala +++ b/spark/src/test/spark-3.4/org/apache/comet/exec/CometExec3_4Suite.scala @@ -43,7 +43,7 @@ class CometExec3_4Suite extends CometTestBase { // The syntax is only supported by Spark 3.4+. test("subquery limit: limit with offset should return correct results") { - withSQLConf(CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { withTable("t1", "t2") { val table1 = """create temporary view t1 as select * from values @@ -95,7 +95,7 @@ class CometExec3_4Suite extends CometTestBase { // Dataset.offset API is not available before Spark 3.4 test("offset") { - withSQLConf(CometConf.COMET_COLUMNAR_SHUFFLE_ENABLED.key -> "true") { + withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { checkSparkAnswer(testData.offset(90)) checkSparkAnswer(arrayData.toDF().offset(99)) checkSparkAnswer(mapData.toDF().offset(99)) From 9125e6a04dce7c86bb0e4f4f297c8c2b70a39559 Mon Sep 17 00:00:00 2001 From: advancedxy Date: Fri, 24 May 2024 07:00:16 +0800 Subject: [PATCH 09/14] feat: Add random row generator in data generator (#451) * feat: Add random row generator in data gen * fix * remove array type match case, which should already been handled in RandomDataGenerator.forType * fix style issue * address comments --- .../org/apache/comet/DataGenerator.scala | 41 ++++++++++++++++ .../org/apache/comet/DataGeneratorSuite.scala | 49 +++++++++++++++++++ 2 files changed, 90 insertions(+) create mode 100644 spark/src/test/scala/org/apache/comet/DataGeneratorSuite.scala diff --git a/spark/src/test/scala/org/apache/comet/DataGenerator.scala b/spark/src/test/scala/org/apache/comet/DataGenerator.scala index 691a371b5..80e7c2288 100644 --- a/spark/src/test/scala/org/apache/comet/DataGenerator.scala +++ b/spark/src/test/scala/org/apache/comet/DataGenerator.scala @@ -21,14 +21,20 @@ package org.apache.comet import scala.util.Random +import org.apache.spark.sql.{RandomDataGenerator, Row} +import org.apache.spark.sql.types.{StringType, StructType} + object DataGenerator { // note that we use `def` rather than `val` intentionally here so that // each test suite starts with a fresh data generator to help ensure // that tests are deterministic def DEFAULT = new DataGenerator(new Random(42)) + // matches the probability of nulls in Spark's RandomDataGenerator + private val PROBABILITY_OF_NULL: Float = 0.1f } class DataGenerator(r: Random) { + import DataGenerator._ /** Generate a random string using the specified characters */ def generateString(chars: String, maxLen: Int): String = { @@ -95,4 +101,39 @@ class DataGenerator(r: Random) { Range(0, n).map(_ => r.nextLong()) } + // Generate a random row according to the schema, the string filed in the struct could be + // configured to generate strings by passing a stringGen function. Other types are delegated + // to Spark's RandomDataGenerator. + def generateRow(schema: StructType, stringGen: Option[() => String] = None): Row = { + val fields = schema.fields.map { f => + f.dataType match { + case StructType(children) => + generateRow(StructType(children), stringGen) + case StringType if stringGen.isDefined => + val gen = stringGen.get + val data = if (f.nullable && r.nextFloat() <= PROBABILITY_OF_NULL) { + null + } else { + gen() + } + data + case _ => + val gen = RandomDataGenerator.forType(f.dataType, f.nullable, r) match { + case Some(g) => g + case None => + throw new IllegalStateException(s"No RandomDataGenerator for type ${f.dataType}") + } + gen() + } + }.toSeq + Row.fromSeq(fields) + } + + def generateRows( + num: Int, + schema: StructType, + stringGen: Option[() => String] = None): Seq[Row] = { + Range(0, num).map(_ => generateRow(schema, stringGen)) + } + } diff --git a/spark/src/test/scala/org/apache/comet/DataGeneratorSuite.scala b/spark/src/test/scala/org/apache/comet/DataGeneratorSuite.scala new file mode 100644 index 000000000..02dfb9d7b --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/DataGeneratorSuite.scala @@ -0,0 +1,49 @@ +/* + * 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. + */ + +package org.apache.comet + +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.types.StructType + +class DataGeneratorSuite extends CometTestBase { + + test("test configurable stringGen in row generator") { + val gen = DataGenerator.DEFAULT + val chars = "abcde" + val maxLen = 10 + val stringGen = () => gen.generateString(chars, maxLen) + val numRows = 100 + val schema = new StructType().add("a", "string") + var numNulls = 0 + gen + .generateRows(numRows, schema, Some(stringGen)) + .foreach(row => { + if (row.getString(0) != null) { + assert(row.getString(0).forall(chars.toSeq.contains)) + assert(row.getString(0).length <= maxLen) + } else { + numNulls += 1 + } + }) + // 0.1 null probability + assert(numNulls >= 0.05 * numRows && numNulls <= 0.15 * numRows) + } + +} From 93af70438b92049226dfd130e04dd83a9863f1a9 Mon Sep 17 00:00:00 2001 From: advancedxy Date: Fri, 24 May 2024 23:13:24 +0800 Subject: [PATCH 10/14] fix: Compute murmur3 hash with dictionary input correctly (#433) * fix: Handle compute murmur3 hash with dictionary input correctly * add unit tests * spotless apply * apply scala fix * address comment * another style issue * Update spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala Co-authored-by: Liang-Chi Hsieh --------- Co-authored-by: Liang-Chi Hsieh --- core/src/execution/datafusion/spark_hash.rs | 270 ++++++++---------- .../apache/comet/CometExpressionSuite.scala | 58 +++- 2 files changed, 163 insertions(+), 165 deletions(-) diff --git a/core/src/execution/datafusion/spark_hash.rs b/core/src/execution/datafusion/spark_hash.rs index aa4269dd0..6d25a72f6 100644 --- a/core/src/execution/datafusion/spark_hash.rs +++ b/core/src/execution/datafusion/spark_hash.rs @@ -17,7 +17,10 @@ //! This includes utilities for hashing and murmur3 hashing. -use arrow::datatypes::{ArrowNativeTypeOp, UInt16Type, UInt32Type, UInt64Type, UInt8Type}; +use arrow::{ + compute::take, + datatypes::{ArrowNativeTypeOp, UInt16Type, UInt32Type, UInt64Type, UInt8Type}, +}; use std::sync::Arc; use datafusion::{ @@ -95,19 +98,8 @@ pub(crate) fn spark_compatible_murmur3_hash>(data: T, seed: u32) } } -#[test] -fn test_murmur3() { - let _hashes = ["", "a", "ab", "abc", "abcd", "abcde"] - .into_iter() - .map(|s| spark_compatible_murmur3_hash(s.as_bytes(), 42) as i32) - .collect::>(); - let _expected = vec![ - 142593372, 1485273170, -97053317, 1322437556, -396302900, 814637928, - ]; -} - macro_rules! hash_array { - ($array_type:ident, $column: ident, $hashes: ident) => { + ($array_type: ident, $column: ident, $hashes: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); if array.null_count() == 0 { for (i, hash) in $hashes.iter_mut().enumerate() { @@ -123,8 +115,31 @@ macro_rules! hash_array { }; } +macro_rules! hash_array_boolean { + ($array_type: ident, $column: ident, $hash_input_type: ident, $hashes: ident) => { + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + if array.null_count() == 0 { + for (i, hash) in $hashes.iter_mut().enumerate() { + *hash = spark_compatible_murmur3_hash( + $hash_input_type::from(array.value(i)).to_le_bytes(), + *hash, + ); + } + } else { + for (i, hash) in $hashes.iter_mut().enumerate() { + if !array.is_null(i) { + *hash = spark_compatible_murmur3_hash( + $hash_input_type::from(array.value(i)).to_le_bytes(), + *hash, + ); + } + } + } + }; +} + macro_rules! hash_array_primitive { - ($array_type:ident, $column: ident, $ty: ident, $hashes: ident) => { + ($array_type: ident, $column: ident, $ty: ident, $hashes: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); let values = array.values(); @@ -143,7 +158,7 @@ macro_rules! hash_array_primitive { } macro_rules! hash_array_primitive_float { - ($array_type:ident, $column: ident, $ty: ident, $ty2: ident, $hashes: ident) => { + ($array_type: ident, $column: ident, $ty: ident, $ty2: ident, $hashes: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); let values = array.values(); @@ -172,7 +187,7 @@ macro_rules! hash_array_primitive_float { } macro_rules! hash_array_decimal { - ($array_type:ident, $column: ident, $hashes: ident) => { + ($array_type: ident, $column: ident, $hashes: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); if array.null_count() == 0 { @@ -193,27 +208,33 @@ macro_rules! hash_array_decimal { fn create_hashes_dictionary( array: &ArrayRef, hashes_buffer: &mut [u32], + first_col: bool, ) -> Result<()> { let dict_array = array.as_any().downcast_ref::>().unwrap(); - - // Hash each dictionary value once, and then use that computed - // hash for each key value to avoid a potentially expensive - // redundant hashing for large dictionary elements (e.g. strings) - let dict_values = Arc::clone(dict_array.values()); - let mut dict_hashes = vec![0; dict_values.len()]; - create_hashes(&[dict_values], &mut dict_hashes)?; - - for (hash, key) in hashes_buffer.iter_mut().zip(dict_array.keys().iter()) { - if let Some(key) = key { - let idx = key.to_usize().ok_or_else(|| { - DataFusionError::Internal(format!( - "Can not convert key value {:?} to usize in dictionary of type {:?}", - key, - dict_array.data_type() - )) - })?; - *hash = dict_hashes[idx] - } // no update for Null, consistent with other hashes + if !first_col { + // unpack the dictionary array as each row may have a different hash input + let unpacked = take(dict_array.values().as_ref(), dict_array.keys(), None)?; + create_hashes(&[unpacked], hashes_buffer)?; + } else { + // For the first column, hash each dictionary value once, and then use + // that computed hash for each key value to avoid a potentially + // expensive redundant hashing for large dictionary elements (e.g. strings) + let dict_values = Arc::clone(dict_array.values()); + // same initial seed as Spark + let mut dict_hashes = vec![42; dict_values.len()]; + create_hashes(&[dict_values], &mut dict_hashes)?; + for (hash, key) in hashes_buffer.iter_mut().zip(dict_array.keys().iter()) { + if let Some(key) = key { + let idx = key.to_usize().ok_or_else(|| { + DataFusionError::Internal(format!( + "Can not convert key value {:?} to usize in dictionary of type {:?}", + key, + dict_array.data_type() + )) + })?; + *hash = dict_hashes[idx] + } // no update for Null, consistent with other hashes + } } Ok(()) } @@ -227,27 +248,11 @@ pub fn create_hashes<'a>( arrays: &[ArrayRef], hashes_buffer: &'a mut [u32], ) -> Result<&'a mut [u32]> { - for col in arrays { + for (i, col) in arrays.iter().enumerate() { + let first_col = i == 0; match col.data_type() { DataType::Boolean => { - let array = col.as_any().downcast_ref::().unwrap(); - if array.null_count() == 0 { - for (i, hash) in hashes_buffer.iter_mut().enumerate() { - *hash = spark_compatible_murmur3_hash( - i32::from(array.value(i)).to_le_bytes(), - *hash, - ); - } - } else { - for (i, hash) in hashes_buffer.iter_mut().enumerate() { - if !array.is_null(i) { - *hash = spark_compatible_murmur3_hash( - i32::from(array.value(i)).to_le_bytes(), - *hash, - ); - } - } - } + hash_array_boolean!(BooleanArray, col, i32, hashes_buffer); } DataType::Int8 => { hash_array_primitive!(Int8Array, col, i32, hashes_buffer); @@ -305,28 +310,28 @@ pub fn create_hashes<'a>( } DataType::Dictionary(index_type, _) => match **index_type { DataType::Int8 => { - create_hashes_dictionary::(col, hashes_buffer)?; + create_hashes_dictionary::(col, hashes_buffer, first_col)?; } DataType::Int16 => { - create_hashes_dictionary::(col, hashes_buffer)?; + create_hashes_dictionary::(col, hashes_buffer, first_col)?; } DataType::Int32 => { - create_hashes_dictionary::(col, hashes_buffer)?; + create_hashes_dictionary::(col, hashes_buffer, first_col)?; } DataType::Int64 => { - create_hashes_dictionary::(col, hashes_buffer)?; + create_hashes_dictionary::(col, hashes_buffer, first_col)?; } DataType::UInt8 => { - create_hashes_dictionary::(col, hashes_buffer)?; + create_hashes_dictionary::(col, hashes_buffer, first_col)?; } DataType::UInt16 => { - create_hashes_dictionary::(col, hashes_buffer)?; + create_hashes_dictionary::(col, hashes_buffer, first_col)?; } DataType::UInt32 => { - create_hashes_dictionary::(col, hashes_buffer)?; + create_hashes_dictionary::(col, hashes_buffer, first_col)?; } DataType::UInt64 => { - create_hashes_dictionary::(col, hashes_buffer)?; + create_hashes_dictionary::(col, hashes_buffer, first_col)?; } _ => { return Err(DataFusionError::Internal(format!( @@ -363,78 +368,64 @@ mod tests { use crate::execution::datafusion::spark_hash::{create_hashes, pmod}; use datafusion::arrow::array::{ArrayRef, Int32Array, Int64Array, Int8Array, StringArray}; - macro_rules! test_hashes { - ($ty:ty, $values:expr, $expected:expr) => { - let i = Arc::new(<$ty>::from($values)) as ArrayRef; - let mut hashes = vec![42; $values.len()]; + macro_rules! test_hashes_internal { + ($input: expr, $len: expr, $expected: expr) => { + let i = $input as ArrayRef; + let mut hashes = vec![42; $len]; create_hashes(&[i], &mut hashes).unwrap(); assert_eq!(hashes, $expected); }; } + fn test_murmur3_hash>> + 'static>( + values: Vec>, + expected: Vec, + ) { + // copied before inserting nulls + let mut input_with_nulls = values.clone(); + let mut expected_with_nulls = expected.clone(); + let len = values.len(); + let i = Arc::new(T::from(values)) as ArrayRef; + test_hashes_internal!(i, len, expected); + + // test with nulls + let median = len / 2; + input_with_nulls.insert(0, None); + input_with_nulls.insert(median, None); + expected_with_nulls.insert(0, 42); + expected_with_nulls.insert(median, 42); + let with_nulls_len = len + 2; + let nullable_input = Arc::new(T::from(input_with_nulls)) as ArrayRef; + test_hashes_internal!(nullable_input, with_nulls_len, expected_with_nulls); + } + #[test] fn test_i8() { - test_hashes!( - Int8Array, + test_murmur3_hash::( vec![Some(1), Some(0), Some(-1), Some(i8::MAX), Some(i8::MIN)], - vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x43b4d8ed, 0x422a1365] - ); - // with null input - test_hashes!( - Int8Array, - vec![Some(1), None, Some(-1), Some(i8::MAX), Some(i8::MIN)], - vec![0xdea578e3, 42, 0xa0590e3d, 0x43b4d8ed, 0x422a1365] + vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x43b4d8ed, 0x422a1365], ); } #[test] fn test_i32() { - test_hashes!( - Int32Array, + test_murmur3_hash::( vec![Some(1), Some(0), Some(-1), Some(i32::MAX), Some(i32::MIN)], - vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x07fb67e7, 0x2b1f0fc6] - ); - // with null input - test_hashes!( - Int32Array, - vec![ - Some(1), - Some(0), - Some(-1), - None, - Some(i32::MAX), - Some(i32::MIN) - ], - vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 42, 0x07fb67e7, 0x2b1f0fc6] + vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x07fb67e7, 0x2b1f0fc6], ); } #[test] fn test_i64() { - test_hashes!( - Int64Array, + test_murmur3_hash::( vec![Some(1), Some(0), Some(-1), Some(i64::MAX), Some(i64::MIN)], - vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb] - ); - // with null input - test_hashes!( - Int64Array, - vec![ - Some(1), - Some(0), - Some(-1), - None, - Some(i64::MAX), - Some(i64::MIN) - ], - vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 42, 0xa05b5d7b, 0xcd1e64fb] + vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb], ); } #[test] fn test_f32() { - test_hashes!( - Float32Array, + test_murmur3_hash::( vec![ Some(1.0), Some(0.0), @@ -443,28 +434,15 @@ mod tests { Some(99999999999.99999999999), Some(-99999999999.99999999999), ], - vec![0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 0xcbdc340f, 0xc0361c86] - ); - // with null input - test_hashes!( - Float32Array, vec![ - Some(1.0), - Some(0.0), - Some(-0.0), - Some(-1.0), - None, - Some(99999999999.99999999999), - Some(-99999999999.99999999999) + 0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 0xcbdc340f, 0xc0361c86, ], - vec![0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 42, 0xcbdc340f, 0xc0361c86] ); } #[test] fn test_f64() { - test_hashes!( - Float64Array, + test_murmur3_hash::( vec![ Some(1.0), Some(0.0), @@ -473,44 +451,26 @@ mod tests { Some(99999999999.99999999999), Some(-99999999999.99999999999), ], - vec![0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 0xb87e1595, 0xa0eef9f9] - ); - // with null input - test_hashes!( - Float64Array, vec![ - Some(1.0), - Some(0.0), - Some(-0.0), - Some(-1.0), - None, - Some(99999999999.99999999999), - Some(-99999999999.99999999999) + 0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 0xb87e1595, 0xa0eef9f9, ], - vec![0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 42, 0xb87e1595, 0xa0eef9f9] ); } #[test] fn test_str() { - test_hashes!( - StringArray, - vec!["hello", "bar", "", "😁", "天地"], - vec![3286402344, 2486176763, 142593372, 885025535, 2395000894] - ); - // test with null input - test_hashes!( - StringArray, - vec![ - Some("hello"), - Some("bar"), - None, - Some(""), - Some("😁"), - Some("天地") - ], - vec![3286402344, 2486176763, 42, 142593372, 885025535, 2395000894] - ); + let input = vec![ + "hello", "bar", "", "😁", "天地", "a", "ab", "abc", "abcd", "abcde", + ] + .iter() + .map(|s| Some(s.to_string())) + .collect::>>(); + let expected: Vec = vec![ + 3286402344, 2486176763, 142593372, 885025535, 2395000894, 1485273170, 0xfa37157b, + 1322437556, 0xe860e5cc, 814637928, + ]; + + test_murmur3_hash::(input, expected); } #[test] diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 6ca4baf60..992615084 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -1454,17 +1454,55 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { withTable(table) { sql(s"create table $table(col string, a int, b float) using parquet") sql(s""" - |insert into $table values - |('Spark SQL ', 10, 1.2), (NULL, NULL, NULL), ('', 0, 0.0), ('苹果手机', NULL, 3.999999) - |, ('Spark SQL ', 10, 1.2), (NULL, NULL, NULL), ('', 0, 0.0), ('苹果手机', NULL, 3.999999) - |""".stripMargin) + |insert into $table values + |('Spark SQL ', 10, 1.2), (NULL, NULL, NULL), ('', 0, 0.0), ('苹果手机', NULL, 3.999999) + |, ('Spark SQL ', 10, 1.2), (NULL, NULL, NULL), ('', 0, 0.0), ('苹果手机', NULL, 3.999999) + |""".stripMargin) checkSparkAnswerAndOperator(""" - |select - |md5(col), md5(cast(a as string)), md5(cast(b as string)), - |hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col), - |sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128) - |from test - |""".stripMargin) + |select + |md5(col), md5(cast(a as string)), md5(cast(b as string)), + |hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col), + |sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128) + |from test + |""".stripMargin) + } + } + } + } + + test("hash functions with random input") { + val dataGen = DataGenerator.DEFAULT + // sufficient number of rows to create dictionary encoded ArrowArray. + val randomNumRows = 1000 + + val whitespaceChars = " \t\r\n" + val timestampPattern = "0123456789/:T" + whitespaceChars + Seq(true, false).foreach { dictionary => + withSQLConf( + "parquet.enable.dictionary" -> dictionary.toString, + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + val table = "test" + withTable(table) { + sql(s"create table $table(col string, a int, b float) using parquet") + // TODO: Add a Row generator in the data gen class and replace th following code + val col = dataGen.generateStrings(randomNumRows, timestampPattern, 6) + val colA = dataGen.generateInts(randomNumRows) + val colB = dataGen.generateFloats(randomNumRows) + val data = col.zip(colA).zip(colB).map { case ((a, b), c) => (a, b, c) } + data + .toDF("col", "a", "b") + .write + .mode("append") + .insertInto(table) + // with random generated data + // disable cast(b as string) for now, as the cast from float to string may produce incompatible result + checkSparkAnswerAndOperator(""" + |select + |md5(col), md5(cast(a as string)), --md5(cast(b as string)), + |hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col), + |sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128) + |from test + |""".stripMargin) } } } From 79431f8837f21f91cb24f3a0b4c11a364dcd70f9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 25 May 2024 12:48:23 -0600 Subject: [PATCH 11/14] fix: Only delegate to DataFusion cast when we know that it is compatible with Spark (#461) * only delegate to DataFusion cast when we know that it is compatible with Spark * add more supported casts * improve support for dictionary-encoded string arrays * clippy * fix merge conflict * fix a regression * fix a regression * fix a regression * fix regression * fix regression * fix regression * remove TODO comment now that issue has been filed * remove cast int32/int64 -> decimal from datafusion compatible list * Revert "remove cast int32/int64 -> decimal from datafusion compatible list" This reverts commit 340e00007575e7f91affa018caa7cd9e2d8964f9. * add comment --- .../execution/datafusion/expressions/cast.rs | 182 +++++++++++------- 1 file changed, 115 insertions(+), 67 deletions(-) diff --git a/core/src/execution/datafusion/expressions/cast.rs b/core/src/execution/datafusion/expressions/cast.rs index fd1f9166d..7e8a96f28 100644 --- a/core/src/execution/datafusion/expressions/cast.rs +++ b/core/src/execution/datafusion/expressions/cast.rs @@ -503,41 +503,37 @@ impl Cast { fn cast_array(&self, array: ArrayRef) -> DataFusionResult { let to_type = &self.data_type; let array = array_with_timezone(array, self.timezone.clone(), Some(to_type)); + let from_type = array.data_type().clone(); + + // unpack dictionary string arrays first + // TODO: we are unpacking a dictionary-encoded array and then performing + // the cast. We could potentially improve performance here by casting the + // dictionary values directly without unpacking the array first, although this + // would add more complexity to the code + let array = match &from_type { + DataType::Dictionary(key_type, value_type) + if key_type.as_ref() == &DataType::Int32 + && (value_type.as_ref() == &DataType::Utf8 + || value_type.as_ref() == &DataType::LargeUtf8) => + { + cast_with_options(&array, value_type.as_ref(), &CAST_OPTIONS)? + } + _ => array, + }; let from_type = array.data_type(); + let cast_result = match (from_type, to_type) { (DataType::Utf8, DataType::Boolean) => { - Self::spark_cast_utf8_to_boolean::(&array, self.eval_mode)? + Self::spark_cast_utf8_to_boolean::(&array, self.eval_mode) } (DataType::LargeUtf8, DataType::Boolean) => { - Self::spark_cast_utf8_to_boolean::(&array, self.eval_mode)? + Self::spark_cast_utf8_to_boolean::(&array, self.eval_mode) } (DataType::Utf8, DataType::Timestamp(_, _)) => { - Self::cast_string_to_timestamp(&array, to_type, self.eval_mode)? + Self::cast_string_to_timestamp(&array, to_type, self.eval_mode) } (DataType::Utf8, DataType::Date32) => { - Self::cast_string_to_date(&array, to_type, self.eval_mode)? - } - (DataType::Dictionary(key_type, value_type), DataType::Date32) - if key_type.as_ref() == &DataType::Int32 - && (value_type.as_ref() == &DataType::Utf8 - || value_type.as_ref() == &DataType::LargeUtf8) => - { - match value_type.as_ref() { - DataType::Utf8 => { - let unpacked_array = - cast_with_options(&array, &DataType::Utf8, &CAST_OPTIONS)?; - Self::cast_string_to_date(&unpacked_array, to_type, self.eval_mode)? - } - DataType::LargeUtf8 => { - let unpacked_array = - cast_with_options(&array, &DataType::LargeUtf8, &CAST_OPTIONS)?; - Self::cast_string_to_date(&unpacked_array, to_type, self.eval_mode)? - } - dt => unreachable!( - "{}", - format!("invalid value type {dt} for dictionary-encoded string array") - ), - } + Self::cast_string_to_date(&array, to_type, self.eval_mode) } (DataType::Int64, DataType::Int32) | (DataType::Int64, DataType::Int16) @@ -547,61 +543,33 @@ impl Cast { | (DataType::Int16, DataType::Int8) if self.eval_mode != EvalMode::Try => { - Self::spark_cast_int_to_int(&array, self.eval_mode, from_type, to_type)? + Self::spark_cast_int_to_int(&array, self.eval_mode, from_type, to_type) } ( DataType::Utf8, DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64, - ) => Self::cast_string_to_int::(to_type, &array, self.eval_mode)?, + ) => Self::cast_string_to_int::(to_type, &array, self.eval_mode), ( DataType::LargeUtf8, DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64, - ) => Self::cast_string_to_int::(to_type, &array, self.eval_mode)?, - ( - DataType::Dictionary(key_type, value_type), - DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64, - ) if key_type.as_ref() == &DataType::Int32 - && (value_type.as_ref() == &DataType::Utf8 - || value_type.as_ref() == &DataType::LargeUtf8) => - { - // TODO: we are unpacking a dictionary-encoded array and then performing - // the cast. We could potentially improve performance here by casting the - // dictionary values directly without unpacking the array first, although this - // would add more complexity to the code - match value_type.as_ref() { - DataType::Utf8 => { - let unpacked_array = - cast_with_options(&array, &DataType::Utf8, &CAST_OPTIONS)?; - Self::cast_string_to_int::(to_type, &unpacked_array, self.eval_mode)? - } - DataType::LargeUtf8 => { - let unpacked_array = - cast_with_options(&array, &DataType::LargeUtf8, &CAST_OPTIONS)?; - Self::cast_string_to_int::(to_type, &unpacked_array, self.eval_mode)? - } - dt => unreachable!( - "{}", - format!("invalid value type {dt} for dictionary-encoded string array") - ), - } - } + ) => Self::cast_string_to_int::(to_type, &array, self.eval_mode), (DataType::Float64, DataType::Utf8) => { - Self::spark_cast_float64_to_utf8::(&array, self.eval_mode)? + Self::spark_cast_float64_to_utf8::(&array, self.eval_mode) } (DataType::Float64, DataType::LargeUtf8) => { - Self::spark_cast_float64_to_utf8::(&array, self.eval_mode)? + Self::spark_cast_float64_to_utf8::(&array, self.eval_mode) } (DataType::Float32, DataType::Utf8) => { - Self::spark_cast_float32_to_utf8::(&array, self.eval_mode)? + Self::spark_cast_float32_to_utf8::(&array, self.eval_mode) } (DataType::Float32, DataType::LargeUtf8) => { - Self::spark_cast_float32_to_utf8::(&array, self.eval_mode)? + Self::spark_cast_float32_to_utf8::(&array, self.eval_mode) } (DataType::Float32, DataType::Decimal128(precision, scale)) => { - Self::cast_float32_to_decimal128(&array, *precision, *scale, self.eval_mode)? + Self::cast_float32_to_decimal128(&array, *precision, *scale, self.eval_mode) } (DataType::Float64, DataType::Decimal128(precision, scale)) => { - Self::cast_float64_to_decimal128(&array, *precision, *scale, self.eval_mode)? + Self::cast_float64_to_decimal128(&array, *precision, *scale, self.eval_mode) } (DataType::Float32, DataType::Int8) | (DataType::Float32, DataType::Int16) @@ -622,14 +590,94 @@ impl Cast { self.eval_mode, from_type, to_type, - )? + ) + } + _ if Self::is_datafusion_spark_compatible(from_type, to_type) => { + // use DataFusion cast only when we know that it is compatible with Spark + Ok(cast_with_options(&array, to_type, &CAST_OPTIONS)?) } _ => { - // when we have no Spark-specific casting we delegate to DataFusion - cast_with_options(&array, to_type, &CAST_OPTIONS)? + // we should never reach this code because the Scala code should be checking + // for supported cast operations and falling back to Spark for anything that + // is not yet supported + Err(CometError::Internal(format!( + "Native cast invoked for unsupported cast from {from_type:?} to {to_type:?}" + ))) } }; - Ok(spark_cast(cast_result, from_type, to_type)) + Ok(spark_cast(cast_result?, from_type, to_type)) + } + + /// Determines if DataFusion supports the given cast in a way that is + /// compatible with Spark + fn is_datafusion_spark_compatible(from_type: &DataType, to_type: &DataType) -> bool { + if from_type == to_type { + return true; + } + match from_type { + DataType::Boolean => matches!( + to_type, + DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Utf8 + ), + DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64 => { + // note that the cast from Int32/Int64 -> Decimal128 here is actually + // not compatible with Spark (no overflow checks) but we have tests that + // rely on this cast working so we have to leave it here for now + matches!( + to_type, + DataType::Boolean + | DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Decimal128(_, _) + | DataType::Utf8 + ) + } + DataType::Float32 | DataType::Float64 => matches!( + to_type, + DataType::Boolean + | DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + ), + DataType::Decimal128(_, _) | DataType::Decimal256(_, _) => matches!( + to_type, + DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Decimal128(_, _) + | DataType::Decimal256(_, _) + ), + DataType::Utf8 => matches!(to_type, DataType::Binary), + DataType::Date32 => matches!(to_type, DataType::Utf8), + DataType::Timestamp(_, _) => { + matches!( + to_type, + DataType::Int64 | DataType::Date32 | DataType::Utf8 | DataType::Timestamp(_, _) + ) + } + DataType::Binary => { + // note that this is not completely Spark compatible because + // DataFusion only supports binary data containing valid UTF-8 strings + matches!(to_type, DataType::Utf8) + } + _ => false, + } } fn cast_string_to_int( From 5bfe46d4f1841985a85b74dc8e8ceaf10a8e3266 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 25 May 2024 14:05:02 -0700 Subject: [PATCH 12/14] fix: `ColumnReader.loadVector` should initiate `CometDictionary` after re-import arrays (#473) --- .../org/apache/comet/parquet/ColumnReader.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java index 46fd87f6b..50991d5b0 100644 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -230,15 +230,15 @@ public CometDecodedVector loadVector() { // return plain vector. currentVector = cometVector; return currentVector; - } else if (dictionary == null) { - // There is dictionary from native side but the Java side dictionary hasn't been - // initialized yet. - Dictionary arrowDictionary = dictionaryProvider.lookup(dictionaryEncoding.getId()); - CometPlainVector dictionaryVector = - new CometPlainVector(arrowDictionary.getVector(), useDecimal128, isUuid); - dictionary = new CometDictionary(dictionaryVector); } + // We should already re-initiate `CometDictionary` here because `Data.importVector` API will + // release the previous dictionary vector and create a new one. + Dictionary arrowDictionary = dictionaryProvider.lookup(dictionaryEncoding.getId()); + CometPlainVector dictionaryVector = + new CometPlainVector(arrowDictionary.getVector(), useDecimal128, isUuid); + dictionary = new CometDictionary(dictionaryVector); + currentVector = new CometDictionaryVector( cometVector, dictionary, dictionaryProvider, useDecimal128, false, isUuid); From 7ba569357cadade64b49b6bdf5f0946f81f95301 Mon Sep 17 00:00:00 2001 From: Son <14060682+sonhmai@users.noreply.github.com> Date: Mon, 27 May 2024 03:13:19 +0700 Subject: [PATCH 13/14] fix: substring with negative indices should produce correct result (#470) --- core/src/execution/datafusion/planner.rs | 3 ++- .../test/scala/org/apache/comet/CometExpressionSuite.scala | 6 ++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index 01d892381..7a37e3aae 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -395,7 +395,8 @@ impl PhysicalPlanner { let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; // Spark Substring's start is 1-based when start > 0 let start = expr.start - i32::from(expr.start > 0); - let len = expr.len; + // substring negative len is treated as 0 in Spark + let len = std::cmp::max(expr.len, 0); Ok(Arc::new(SubstringExec::new( child, diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 992615084..1afdd78ec 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -156,6 +156,12 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("string type and substring") { withParquetTable((0 until 5).map(i => (i.toString, (i + 100).toString)), "tbl") { checkSparkAnswerAndOperator("SELECT _1, substring(_2, 2, 2) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, substring(_2, 2, -2) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, substring(_2, -2, 2) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, substring(_2, -2, -2) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, substring(_2, -2, 10) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, substring(_2, 0, 0) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, substring(_2, 1, 0) FROM tbl") } } From 479a97a7a40f13609d591a5a08c1265b1cc74fdd Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 27 May 2024 18:55:19 -0700 Subject: [PATCH 14/14] fix: CometReader.loadVector should not overwrite dictionary ids (#476) * fix: CometReader.loadVector should not overwrite dictionary ids * For review --- .../apache/arrow/c/CometSchemaImporter.java | 73 +++++++++++++++++++ .../org/apache/comet/parquet/BatchReader.java | 13 ++++ .../apache/comet/parquet/ColumnReader.java | 23 +++--- .../comet/parquet/LazyColumnReader.java | 4 +- .../java/org/apache/comet/parquet/Utils.java | 10 ++- 5 files changed, 107 insertions(+), 16 deletions(-) create mode 100644 common/src/main/java/org/apache/arrow/c/CometSchemaImporter.java diff --git a/common/src/main/java/org/apache/arrow/c/CometSchemaImporter.java b/common/src/main/java/org/apache/arrow/c/CometSchemaImporter.java new file mode 100644 index 000000000..32955f1ac --- /dev/null +++ b/common/src/main/java/org/apache/arrow/c/CometSchemaImporter.java @@ -0,0 +1,73 @@ +/* + * 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. + */ + +package org.apache.arrow.c; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.types.pojo.Field; + +/** This is a simple wrapper around SchemaImporter to make it accessible from Java Arrow. */ +public class CometSchemaImporter { + private final BufferAllocator allocator; + private final SchemaImporter importer; + private final CDataDictionaryProvider provider = new CDataDictionaryProvider(); + + public CometSchemaImporter(BufferAllocator allocator) { + this.allocator = allocator; + this.importer = new SchemaImporter(allocator); + } + + public BufferAllocator getAllocator() { + return allocator; + } + + public CDataDictionaryProvider getProvider() { + return provider; + } + + public Field importField(ArrowSchema schema) { + try { + return importer.importField(schema, provider); + } finally { + schema.release(); + schema.close(); + } + } + + /** + * Imports data from ArrowArray/ArrowSchema into a FieldVector. This is basically the same as Java + * Arrow `Data.importVector`. `Data.importVector` initiates `SchemaImporter` internally which is + * used to fill dictionary ids for dictionary encoded vectors. Every call to `importVector` will + * begin with dictionary ids starting from 0. So, separate calls to `importVector` will overwrite + * dictionary ids. To avoid this, we need to use the same `SchemaImporter` instance for all calls + * to `importVector`. + */ + public FieldVector importVector(ArrowArray array, ArrowSchema schema) { + Field field = importField(schema); + FieldVector vector = field.createVector(allocator); + Data.importIntoVector(allocator, array, vector, provider); + + return vector; + } + + public void close() { + provider.close(); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/BatchReader.java b/common/src/main/java/org/apache/comet/parquet/BatchReader.java index 9940390dc..bf8e6e550 100644 --- a/common/src/main/java/org/apache/comet/parquet/BatchReader.java +++ b/common/src/main/java/org/apache/comet/parquet/BatchReader.java @@ -37,6 +37,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.arrow.c.CometSchemaImporter; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -88,6 +91,7 @@ */ public class BatchReader extends RecordReader implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(FileReader.class); + protected static final BufferAllocator ALLOCATOR = new RootAllocator(); private Configuration conf; private int capacity; @@ -104,6 +108,7 @@ public class BatchReader extends RecordReader implements Cl private MessageType requestedSchema; private CometVector[] vectors; private AbstractColumnReader[] columnReaders; + private CometSchemaImporter importer; private ColumnarBatch currentBatch; private Future> prefetchTask; private LinkedBlockingQueue> prefetchQueue; @@ -515,6 +520,10 @@ public void close() throws IOException { fileReader.close(); fileReader = null; } + if (importer != null) { + importer.close(); + importer = null; + } } @SuppressWarnings("deprecation") @@ -552,6 +561,9 @@ private boolean loadNextRowGroupIfNecessary() throws Throwable { numRowGroupsMetric.add(1); } + if (importer != null) importer.close(); + importer = new CometSchemaImporter(ALLOCATOR); + List columns = requestedSchema.getColumns(); for (int i = 0; i < columns.size(); i++) { if (missingColumns[i]) continue; @@ -564,6 +576,7 @@ private boolean loadNextRowGroupIfNecessary() throws Throwable { Utils.getColumnReader( dataType, columns.get(i), + importer, capacity, useDecimal128, useLazyMaterialization, diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java index 50991d5b0..3d4cb3aa5 100644 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -27,10 +27,7 @@ import org.apache.arrow.c.ArrowArray; import org.apache.arrow.c.ArrowSchema; -import org.apache.arrow.c.CDataDictionaryProvider; -import org.apache.arrow.c.Data; -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.c.CometSchemaImporter; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.dictionary.Dictionary; import org.apache.arrow.vector.types.pojo.DictionaryEncoding; @@ -53,7 +50,6 @@ public class ColumnReader extends AbstractColumnReader { protected static final Logger LOG = LoggerFactory.getLogger(ColumnReader.class); - protected static final BufferAllocator ALLOCATOR = new RootAllocator(); /** * The current Comet vector holding all the values read by this column reader. Owned by this @@ -89,18 +85,19 @@ public class ColumnReader extends AbstractColumnReader { */ boolean hadNull; - /** Dictionary provider for this column. */ - private final CDataDictionaryProvider dictionaryProvider = new CDataDictionaryProvider(); + private final CometSchemaImporter importer; public ColumnReader( DataType type, ColumnDescriptor descriptor, + CometSchemaImporter importer, int batchSize, boolean useDecimal128, boolean useLegacyDateTimestamp) { super(type, descriptor, useDecimal128, useLegacyDateTimestamp); assert batchSize > 0 : "Batch size must be positive, found " + batchSize; this.batchSize = batchSize; + this.importer = importer; initNative(); } @@ -164,7 +161,6 @@ public void close() { currentVector.close(); currentVector = null; } - dictionaryProvider.close(); super.close(); } @@ -209,10 +205,11 @@ public CometDecodedVector loadVector() { try (ArrowArray array = ArrowArray.wrap(addresses[0]); ArrowSchema schema = ArrowSchema.wrap(addresses[1])) { - FieldVector vector = Data.importVector(ALLOCATOR, array, schema, dictionaryProvider); + FieldVector vector = importer.importVector(array, schema); + DictionaryEncoding dictionaryEncoding = vector.getField().getDictionary(); - CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128, isUuid); + CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128); // Update whether the current vector contains any null values. This is used in the following // batch(s) to determine whether we can skip loading the native vector. @@ -234,15 +231,17 @@ public CometDecodedVector loadVector() { // We should already re-initiate `CometDictionary` here because `Data.importVector` API will // release the previous dictionary vector and create a new one. - Dictionary arrowDictionary = dictionaryProvider.lookup(dictionaryEncoding.getId()); + Dictionary arrowDictionary = importer.getProvider().lookup(dictionaryEncoding.getId()); CometPlainVector dictionaryVector = new CometPlainVector(arrowDictionary.getVector(), useDecimal128, isUuid); dictionary = new CometDictionary(dictionaryVector); currentVector = new CometDictionaryVector( - cometVector, dictionary, dictionaryProvider, useDecimal128, false, isUuid); + cometVector, dictionary, importer.getProvider(), useDecimal128, false, isUuid); + currentVector = + new CometDictionaryVector(cometVector, dictionary, importer.getProvider(), useDecimal128); return currentVector; } } diff --git a/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java b/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java index a15d84192..dd08a88ab 100644 --- a/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java @@ -21,6 +21,7 @@ import java.io.IOException; +import org.apache.arrow.c.CometSchemaImporter; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.page.PageReader; import org.apache.spark.sql.types.DataType; @@ -45,10 +46,11 @@ public class LazyColumnReader extends ColumnReader { public LazyColumnReader( DataType sparkReadType, ColumnDescriptor descriptor, + CometSchemaImporter importer, int batchSize, boolean useDecimal128, boolean useLegacyDateTimestamp) { - super(sparkReadType, descriptor, batchSize, useDecimal128, useLegacyDateTimestamp); + super(sparkReadType, descriptor, importer, batchSize, useDecimal128, useLegacyDateTimestamp); this.batchSize = 0; // the batch size is set later in `readBatch` this.vector = new CometLazyVector(sparkReadType, this, useDecimal128); } diff --git a/common/src/main/java/org/apache/comet/parquet/Utils.java b/common/src/main/java/org/apache/comet/parquet/Utils.java index 95ca06cda..99f3a4edd 100644 --- a/common/src/main/java/org/apache/comet/parquet/Utils.java +++ b/common/src/main/java/org/apache/comet/parquet/Utils.java @@ -19,6 +19,7 @@ package org.apache.comet.parquet; +import org.apache.arrow.c.CometSchemaImporter; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.PrimitiveType; @@ -28,26 +29,29 @@ public class Utils { public static ColumnReader getColumnReader( DataType type, ColumnDescriptor descriptor, + CometSchemaImporter importer, int batchSize, boolean useDecimal128, boolean useLazyMaterialization) { // TODO: support `useLegacyDateTimestamp` for Iceberg return getColumnReader( - type, descriptor, batchSize, useDecimal128, useLazyMaterialization, true); + type, descriptor, importer, batchSize, useDecimal128, useLazyMaterialization, true); } public static ColumnReader getColumnReader( DataType type, ColumnDescriptor descriptor, + CometSchemaImporter importer, int batchSize, boolean useDecimal128, boolean useLazyMaterialization, boolean useLegacyDateTimestamp) { if (useLazyMaterialization && supportLazyMaterialization(type)) { return new LazyColumnReader( - type, descriptor, batchSize, useDecimal128, useLegacyDateTimestamp); + type, descriptor, importer, batchSize, useDecimal128, useLegacyDateTimestamp); } else { - return new ColumnReader(type, descriptor, batchSize, useDecimal128, useLegacyDateTimestamp); + return new ColumnReader( + type, descriptor, importer, batchSize, useDecimal128, useLegacyDateTimestamp); } }