From d59241751a72b335c749c73add4124a665ad1f26 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 May 2024 08:23:00 -0600 Subject: [PATCH 01/38] Only allow supported cast operations to be converted to native --- .../comet/CometSparkSessionExtensions.scala | 8 +- .../apache/comet/expressions/CometCast.scala | 136 ++++++++++++++++++ .../apache/comet/serde/QueryPlanSerde.scala | 18 +-- .../org/apache/comet/CometCastSuite.scala | 6 +- 4 files changed, 149 insertions(+), 19 deletions(-) create mode 100644 spark/src/main/scala/org/apache/comet/expressions/CometCast.scala diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 1e78013c7..0a1292bfc 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -1042,10 +1042,10 @@ object CometSparkSessionExtensions extends Logging { * The node with information (if any) attached */ def withInfo[T <: TreeNode[_]](node: T, info: String, exprs: T*): T = { - val exprInfo = exprs - .flatMap { e => Seq(e.getTagValue(CometExplainInfo.EXTENSION_INFO)) } - .flatten - .mkString("\n") + val exprInfo = + (Seq(node.getTagValue(CometExplainInfo.EXTENSION_INFO)) ++ exprs + .flatMap(e => Seq(e.getTagValue(CometExplainInfo.EXTENSION_INFO)))).flatten + .mkString("\n") if (info != null && info.nonEmpty && exprInfo.nonEmpty) { node.setTagValue(CometExplainInfo.EXTENSION_INFO, Seq(exprInfo, info).mkString("\n")) } else if (exprInfo.nonEmpty) { diff --git a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala new file mode 100644 index 000000000..9ee81eb49 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -0,0 +1,136 @@ +/* + * 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.expressions + +import org.apache.spark.sql.catalyst.expressions.Cast +import org.apache.spark.sql.types.{DataType, DataTypes, DecimalType} + +import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions.withInfo + +object CometCast { + + def isSupported( + cast: Cast, + fromType: DataType, + toType: DataType, + timeZoneId: Option[String], + evalMode: String): Boolean = { + (fromType, toType) match { + case (DataTypes.StringType, _) => + castFromStringSupported(cast, toType, timeZoneId, evalMode) + case (_, DataTypes.StringType) => + castToStringSupported(cast, fromType, timeZoneId, evalMode) + case (DataTypes.TimestampType, _) => + castFromTimestampSupported(cast, toType, timeZoneId, evalMode) + case (_: DecimalType, DataTypes.FloatType | DataTypes.DoubleType) => true + case ( + DataTypes.BooleanType, + DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | + DataTypes.LongType | DataTypes.FloatType | DataTypes.DoubleType) => + true + case ( + DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType, + DataTypes.BooleanType | DataTypes.ByteType | DataTypes.ShortType | + DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | + DataTypes.DoubleType) => + true + case ( + DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType, + _: DecimalType) => + true + case (DataTypes.FloatType, DataTypes.BooleanType | DataTypes.DoubleType) => true + case (DataTypes.DoubleType, DataTypes.BooleanType | DataTypes.FloatType) => true + case _ => false + } + } + + private def castFromStringSupported( + cast: Cast, + toType: DataType, + timeZoneId: Option[String], + evalMode: String): Boolean = { + toType match { + case DataTypes.BooleanType => + true + case DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | + DataTypes.LongType => + true + case DataTypes.FloatType | DataTypes.DoubleType => + // https://github.com/apache/datafusion-comet/issues/326 + false + case _: DecimalType => + // https://github.com/apache/datafusion-comet/issues/325 + false + case DataTypes.DateType => + // https://github.com/apache/datafusion-comet/issues/327 + false + case DataTypes.TimestampType => + val enabled = CometConf.COMET_CAST_STRING_TO_TIMESTAMP.get() + if (!enabled) { + // https://github.com/apache/datafusion-comet/issues/328 + withInfo(cast, s"${CometConf.COMET_CAST_STRING_TO_TIMESTAMP.key} is disabled") + } + enabled + case _ => + false + } + } + + private def castToStringSupported( + cast: Cast, + fromType: DataType, + timeZoneId: Option[String], + evalMode: String): Boolean = { + fromType match { + case DataTypes.BooleanType => true + case DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | + DataTypes.LongType => + true + case DataTypes.DateType => true + case DataTypes.TimestampType => true + case DataTypes.FloatType | DataTypes.DoubleType => + // https://github.com/apache/datafusion-comet/issues/326 + false + case _ => false + } + } + + private def castFromTimestampSupported( + cast: Cast, + toType: DataType, + timeZoneId: Option[String], + evalMode: String): Boolean = { + toType match { + case DataTypes.BooleanType | DataTypes.ByteType | DataTypes.ShortType | + DataTypes.IntegerType => + // https://github.com/apache/datafusion-comet/issues/352 + // this seems like an edge case that isn't important for us to support + false + case DataTypes.LongType => + // https://github.com/apache/datafusion-comet/issues/352 + false + case DataTypes.StringType => true + case DataTypes.DateType => true + case _ => false + } + } + +} 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 6eda0547f..ed7ea0737 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -43,6 +43,7 @@ import org.apache.spark.unsafe.types.UTF8String import org.apache.comet.CometConf import org.apache.comet.CometSparkSessionExtensions.{isCometOperatorEnabled, isCometScan, isSpark32, isSpark34Plus, withInfo} +import org.apache.comet.expressions.CometCast import org.apache.comet.serde.ExprOuterClass.{AggExpr, DataType => ProtoDataType, Expr, ScalarFunc} import org.apache.comet.serde.ExprOuterClass.DataType.{DataTypeInfo, DecimalInfo, ListInfo, MapInfo, StructInfo} import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, JoinType, Operator} @@ -575,7 +576,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { val value = cast.eval() exprToProtoInternal(Literal(value, dataType), inputs) - case Cast(child, dt, timeZoneId, evalMode) => + case cast @ Cast(child, dt, timeZoneId, evalMode) => val childExpr = exprToProtoInternal(child, inputs) if (childExpr.isDefined) { val evalModeStr = if (evalMode.isInstanceOf[Boolean]) { @@ -585,19 +586,12 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { // Spark 3.4+ has EvalMode enum with values LEGACY, ANSI, and TRY evalMode.toString } - val supportedCast = (child.dataType, dt) match { - case (DataTypes.StringType, DataTypes.TimestampType) - if !CometConf.COMET_CAST_STRING_TO_TIMESTAMP.get() => - // https://github.com/apache/datafusion-comet/issues/328 - withInfo(expr, s"${CometConf.COMET_CAST_STRING_TO_TIMESTAMP.key} is disabled") - false - case _ => true - } - if (supportedCast) { + if (CometCast.isSupported(cast, child.dataType, dt, timeZoneId, evalModeStr)) { castToProto(timeZoneId, dt, childExpr, evalModeStr) } else { - // no need to call withInfo here since it was called when determining - // the value for `supportedCast` + withInfo( + expr, + s"Unsupported cast from ${child.dataType} to $dt with timezone $timeZoneId and evalMode $evalModeStr") None } } else { diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 1bddedde9..409cc1620 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -771,12 +771,12 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf((SQLConf.ANSI_ENABLED.key, "false")) { // cast() should return null for invalid inputs when ansi mode is disabled val df = spark.sql(s"select a, cast(a as ${toType.sql}) from t order by a") - checkSparkAnswer(df) + checkSparkAnswerAndOperator(df) // try_cast() should always return null for invalid inputs val df2 = spark.sql(s"select a, try_cast(a as ${toType.sql}) from t order by a") - checkSparkAnswer(df2) + checkSparkAnswerAndOperator(df2) } // with ANSI enabled, we should produce the same exception as Spark @@ -818,7 +818,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { // try_cast() should always return null for invalid inputs val df2 = spark.sql(s"select a, try_cast(a as ${toType.sql}) from t order by a") - checkSparkAnswer(df2) + checkSparkAnswerAndOperator(df2) } } } From 4641540ae4595bac99d57f9a1ad68a97a0898b29 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 May 2024 09:11:54 -0600 Subject: [PATCH 02/38] refactor for readability --- .../apache/comet/expressions/CometCast.scala | 80 +++++++++++-------- 1 file changed, 46 insertions(+), 34 deletions(-) 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 9ee81eb49..51749f1b8 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -35,38 +35,28 @@ object CometCast { evalMode: String): Boolean = { (fromType, toType) match { case (DataTypes.StringType, _) => - castFromStringSupported(cast, toType, timeZoneId, evalMode) + canCastFromString(cast, toType) case (_, DataTypes.StringType) => - castToStringSupported(cast, fromType, timeZoneId, evalMode) + canCastToString(fromType) case (DataTypes.TimestampType, _) => - castFromTimestampSupported(cast, toType, timeZoneId, evalMode) - case (_: DecimalType, DataTypes.FloatType | DataTypes.DoubleType) => true - case ( - DataTypes.BooleanType, - DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | - DataTypes.LongType | DataTypes.FloatType | DataTypes.DoubleType) => - true + canCastFromTimestamp(toType) + case (_: DecimalType, _) => + canCastFromDecimal(toType) + case (DataTypes.BooleanType, _) => + canCastFromBoolean(toType) case ( DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType, - DataTypes.BooleanType | DataTypes.ByteType | DataTypes.ShortType | - DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | - DataTypes.DoubleType) => - true - case ( - DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType, - _: DecimalType) => - true - case (DataTypes.FloatType, DataTypes.BooleanType | DataTypes.DoubleType) => true - case (DataTypes.DoubleType, DataTypes.BooleanType | DataTypes.FloatType) => true + _) => + canCastFromInt(toType) + case (DataTypes.FloatType, _) => + canCastFromFloat(toType) + case (DataTypes.DoubleType, _) => + canCastFromDouble(toType) case _ => false } } - private def castFromStringSupported( - cast: Cast, - toType: DataType, - timeZoneId: Option[String], - evalMode: String): Boolean = { + private def canCastFromString(cast: Cast, toType: DataType): Boolean = { toType match { case DataTypes.BooleanType => true @@ -94,11 +84,7 @@ object CometCast { } } - private def castToStringSupported( - cast: Cast, - fromType: DataType, - timeZoneId: Option[String], - evalMode: String): Boolean = { + private def canCastToString(fromType: DataType): Boolean = { fromType match { case DataTypes.BooleanType => true case DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | @@ -113,11 +99,7 @@ object CometCast { } } - private def castFromTimestampSupported( - cast: Cast, - toType: DataType, - timeZoneId: Option[String], - evalMode: String): Boolean = { + private def canCastFromTimestamp(toType: DataType): Boolean = { toType match { case DataTypes.BooleanType | DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType => @@ -133,4 +115,34 @@ object CometCast { } } + private def canCastFromBoolean(toType: DataType) = toType match { + case DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType | + DataTypes.FloatType | DataTypes.DoubleType => + true + case _ => false + } + + private def canCastFromInt(toType: DataType) = toType match { + case DataTypes.BooleanType | DataTypes.ByteType | DataTypes.ShortType | + DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | DataTypes.DoubleType | + _: DecimalType => + true + case _ => false + } + + private def canCastFromFloat(toType: DataType) = toType match { + case DataTypes.BooleanType | DataTypes.DoubleType => true + case _ => false + } + + private def canCastFromDouble(toType: DataType) = toType match { + case DataTypes.BooleanType | DataTypes.FloatType => true + case _ => false + } + + private def canCastFromDecimal(toType: DataType) = toType match { + case DataTypes.FloatType | DataTypes.DoubleType => true + case _ => false + } + } From 39ac1775ef9ea76d85957ba655d4842dbdba3f64 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 May 2024 09:19:31 -0600 Subject: [PATCH 03/38] Remove unused import --- spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala | 1 - 1 file changed, 1 deletion(-) 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 ed7ea0737..a3b4edb6d 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -41,7 +41,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -import org.apache.comet.CometConf import org.apache.comet.CometSparkSessionExtensions.{isCometOperatorEnabled, isCometScan, isSpark32, isSpark34Plus, withInfo} import org.apache.comet.expressions.CometCast import org.apache.comet.serde.ExprOuterClass.{AggExpr, DataType => ProtoDataType, Expr, ScalarFunc} From c7775a65100214be91bc63a867c8c768eb608848 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 May 2024 10:15:37 -0600 Subject: [PATCH 04/38] formatting --- .../src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 a3b4edb6d..8da71f788 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -590,7 +590,8 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { } else { withInfo( expr, - s"Unsupported cast from ${child.dataType} to $dt with timezone $timeZoneId and evalMode $evalModeStr") + s"Unsupported cast from ${child.dataType} to $dt " + + s"with timezone $timeZoneId and evalMode $evalModeStr") None } } else { From 4e803ccc42fcf1bb28bb06df858f74efa08fd237 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 May 2024 12:59:40 -0600 Subject: [PATCH 05/38] avoid duplicate strings in withInfo --- .../comet/CometSparkSessionExtensions.scala | 23 ++++++++++++------- .../apache/comet/CometExpressionSuite.scala | 22 ++++++++++++++++++ 2 files changed, 37 insertions(+), 8 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 0a1292bfc..8f31561d6 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -1042,14 +1042,21 @@ object CometSparkSessionExtensions extends Logging { * The node with information (if any) attached */ def withInfo[T <: TreeNode[_]](node: T, info: String, exprs: T*): T = { - val exprInfo = - (Seq(node.getTagValue(CometExplainInfo.EXTENSION_INFO)) ++ exprs - .flatMap(e => Seq(e.getTagValue(CometExplainInfo.EXTENSION_INFO)))).flatten - .mkString("\n") - if (info != null && info.nonEmpty && exprInfo.nonEmpty) { - node.setTagValue(CometExplainInfo.EXTENSION_INFO, Seq(exprInfo, info).mkString("\n")) - } else if (exprInfo.nonEmpty) { - node.setTagValue(CometExplainInfo.EXTENSION_INFO, exprInfo) + // TODO maybe we could store the tags as `Set[String]` rather than newline-delimited strings + // and avoid having to split and mkString + val nodeInfo: Set[String] = node + .getTagValue(CometExplainInfo.EXTENSION_INFO) + .map(_.split('\n').toSet) + .getOrElse(Set.empty[String]) + val exprInfo: Set[String] = exprs + .flatMap(e => e.getTagValue(CometExplainInfo.EXTENSION_INFO).map(_.split('\n'))) + .flatten + .toSet + val currentInfo = nodeInfo ++ exprInfo + if (info != null && info.nonEmpty && currentInfo.nonEmpty) { + node.setTagValue(CometExplainInfo.EXTENSION_INFO, (currentInfo + info).mkString("\n")) + } else if (currentInfo.nonEmpty) { + node.setTagValue(CometExplainInfo.EXTENSION_INFO, currentInfo.mkString("\n")) } else if (info != null && info.nonEmpty) { node.setTagValue(CometExplainInfo.EXTENSION_INFO, info) } diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 3683c8d44..a63f459ad 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -21,6 +21,7 @@ package org.apache.comet import org.apache.hadoop.fs.Path import org.apache.spark.sql.{CometTestBase, DataFrame, Row} +import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.expr import org.apache.spark.sql.internal.SQLConf @@ -1427,4 +1428,25 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + test("tag nodes with multiple info") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val project = spark + .sql("SELECT CAST(1 as string)") + .queryExecution + .executedPlan + .asInstanceOf[ProjectExec] + assert(project.getTagValue(CometExplainInfo.EXTENSION_INFO).isEmpty) + CometSparkSessionExtensions.withInfo(project.expressions.head, "expr info 1") + CometSparkSessionExtensions.withInfo(project, "node info 1", project.expressions: _*) + assert( + project.getTagValue(CometExplainInfo.EXTENSION_INFO).get == "expr info 1\nnode info 1") + CometSparkSessionExtensions.withInfo(project, "node info 2", project.expressions: _*) + assert( + project + .getTagValue(CometExplainInfo.EXTENSION_INFO) + .get == "expr info 1\nnode info 1\nnode info 2") + } + + } + } From d2558790e91114d38c3591870f885d4df68f13b2 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 May 2024 15:07:51 -0600 Subject: [PATCH 06/38] always cast between same type --- .../main/scala/org/apache/comet/expressions/CometCast.scala | 5 +++++ 1 file changed, 5 insertions(+) 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 51749f1b8..613d35991 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -33,6 +33,11 @@ object CometCast { toType: DataType, timeZoneId: Option[String], evalMode: String): Boolean = { + + if (fromType == toType) { + return true + } + (fromType, toType) match { case (DataTypes.StringType, _) => canCastFromString(cast, toType) From b2d3d2decb0c9f865014c712d0ecc7249f470932 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 May 2024 19:17:46 -0600 Subject: [PATCH 07/38] save progress --- .../comet/CometSparkSessionExtensions.scala | 3 ++ .../apache/comet/expressions/CometCast.scala | 30 +++++++++++++++++++ .../org/apache/comet/CometCastSuite.scala | 27 +++++++++++++++++ 3 files changed, 60 insertions(+) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 8f31561d6..04f14d9da 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -1042,6 +1042,9 @@ object CometSparkSessionExtensions extends Logging { * The node with information (if any) attached */ def withInfo[T <: TreeNode[_]](node: T, info: String, exprs: T*): T = { + // scalastyle:off println + println("withInfo: " + info) + // scalastyle:on println // TODO maybe we could store the tags as `Set[String]` rather than newline-delimited strings // and avoid having to split and mkString val nodeInfo: Set[String] = node 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 613d35991..86a876f67 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -39,6 +39,34 @@ object CometCast { } (fromType, toType) match { + + // TODO this is a temporary hack to allow casts that we either know are + // incompatible with Spark, or are just not well tested yet, just to avoid + // regressions in existing tests with this PR + + // BEGIN HACK + case (dt: DataType, _) if dt.typeName == "timestamp_ntz" => + toType match { + case DataTypes.TimestampType | DataTypes.DateType | DataTypes.StringType => + true + case _ => false + } + case (_: DecimalType, _: DecimalType) => + // TODO we need to file an issue for adding specific tests for casting + // between decimal types with difference precision and scale + true + case (DataTypes.DoubleType, _: DecimalType) => + true + case (DataTypes.TimestampType, DataTypes.LongType) => + true + case (DataTypes.BinaryType | DataTypes.FloatType, DataTypes.StringType) => + true + case (_, DataTypes.BinaryType) => + true + // END HACK + + case (DataTypes.StringType, DataTypes.TimestampType) => + true case (DataTypes.StringType, _) => canCastFromString(cast, toType) case (_, DataTypes.StringType) => @@ -68,6 +96,8 @@ object CometCast { case DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType => true + case DataTypes.BinaryType => + true case DataTypes.FloatType | DataTypes.DoubleType => // https://github.com/apache/datafusion-comet/issues/326 false diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 409cc1620..ee5f97c86 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -84,6 +84,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { DataTypes.DoubleType, DataTypes.createDecimalType(10, 2), DataTypes.StringType, + DataTypes.BinaryType, DataTypes.DateType, DataTypes.TimestampType) // TODO add DataTypes.TimestampNTZType for Spark 3.4 and later @@ -164,6 +165,10 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateBytes(), DataTypes.StringType) } + ignore("cast ByteType to BinaryType") { + castTest(generateBytes(), DataTypes.BinaryType) + } + ignore("cast ByteType to TimestampType") { // input: -1, expected: 1969-12-31 15:59:59.0, actual: 1969-12-31 15:59:59.999999 castTest(generateBytes(), DataTypes.TimestampType) @@ -204,6 +209,10 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateShorts(), DataTypes.StringType) } + ignore("cast ShortType to BinaryType") { + castTest(generateShorts(), DataTypes.BinaryType) + } + ignore("cast ShortType to TimestampType") { // input: -1003, expected: 1969-12-31 15:43:17.0, actual: 1969-12-31 15:59:59.998997 castTest(generateShorts(), DataTypes.TimestampType) @@ -246,6 +255,10 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateInts(), DataTypes.StringType) } + ignore("cast IntegerType to BinaryType") { + castTest(generateInts(), DataTypes.BinaryType) + } + ignore("cast IntegerType to TimestampType") { // input: -1000479329, expected: 1938-04-19 01:04:31.0, actual: 1969-12-31 15:43:19.520671 castTest(generateInts(), DataTypes.TimestampType) @@ -289,6 +302,10 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateLongs(), DataTypes.StringType) } + ignore("cast LongType to BinaryType") { + castTest(generateLongs(), DataTypes.BinaryType) + } + ignore("cast LongType to TimestampType") { // java.lang.ArithmeticException: long overflow castTest(generateLongs(), DataTypes.TimestampType) @@ -515,6 +532,10 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(values, DataTypes.createDecimalType(10, 2)) } + test("cast StringType to BinaryType") { + castTest(generateStrings(numericPattern, 8).toDF("a"), DataTypes.BinaryType) + } + ignore("cast StringType to DateType") { // https://github.com/apache/datafusion-comet/issues/327 castTest(generateStrings(datePattern, 8).toDF("a"), DataTypes.DateType) @@ -536,6 +557,12 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + // CAST from BinaryType + + ignore("cast BinaryType to StringType") { + // TODO + } + // CAST from DateType ignore("cast DateType to BooleanType") { From 889c754f006c0d71c8dead9aec3c350d5e82ca9b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 08:38:31 -0600 Subject: [PATCH 08/38] skip try_cast testing prior to Spark 3.4 --- .../apache/comet/expressions/CometCast.scala | 8 +++---- .../org/apache/comet/CometCastSuite.scala | 21 +++++++++++++------ 2 files changed, 18 insertions(+), 11 deletions(-) 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 86a876f67..9837d8967 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -51,10 +51,6 @@ object CometCast { true case _ => false } - case (_: DecimalType, _: DecimalType) => - // TODO we need to file an issue for adding specific tests for casting - // between decimal types with difference precision and scale - true case (DataTypes.DoubleType, _: DecimalType) => true case (DataTypes.TimestampType, DataTypes.LongType) => @@ -65,7 +61,9 @@ object CometCast { true // END HACK - case (DataTypes.StringType, DataTypes.TimestampType) => + case (_: DecimalType, _: DecimalType) => + // TODO we need to file an issue for adding specific tests for casting + // between decimal types with different precision and scale true case (DataTypes.StringType, _) => canCastFromString(cast, toType) diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index ee5f97c86..fd108c4c3 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -791,6 +791,11 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } private def castTest(input: DataFrame, toType: DataType): Unit = { + + // we do not support the TryCast expression in Spark 3.2 and 3.3 + // https://github.com/apache/datafusion-comet/issues/374 + val testTryCast = CometSparkSessionExtensions.isSpark34Plus + withTempPath { dir => val data = roundtripParquet(input, dir).coalesce(1) data.createOrReplaceTempView("t") @@ -801,9 +806,11 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { checkSparkAnswerAndOperator(df) // try_cast() should always return null for invalid inputs - val df2 = - spark.sql(s"select a, try_cast(a as ${toType.sql}) from t order by a") - checkSparkAnswerAndOperator(df2) + if (testTryCast) { + val df2 = + spark.sql(s"select a, try_cast(a as ${toType.sql}) from t order by a") + checkSparkAnswerAndOperator(df2) + } } // with ANSI enabled, we should produce the same exception as Spark @@ -843,9 +850,11 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } // try_cast() should always return null for invalid inputs - val df2 = - spark.sql(s"select a, try_cast(a as ${toType.sql}) from t order by a") - checkSparkAnswerAndOperator(df2) + if (testTryCast) { + val df2 = + spark.sql(s"select a, try_cast(a as ${toType.sql}) from t order by a") + checkSparkAnswerAndOperator(df2) + } } } } From 35bcbf9079571230810383b6658cac281267b3a3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 10:10:13 -0600 Subject: [PATCH 09/38] add config to allow incompatible casts --- .../scala/org/apache/comet/CometConf.scala | 12 +- docs/source/user-guide/configs.md | 2 +- .../comet/CometSparkSessionExtensions.scala | 3 - .../apache/comet/expressions/CometCast.scala | 124 +++++++++--------- .../apache/comet/serde/QueryPlanSerde.scala | 25 ++-- .../org/apache/comet/CometCastSuite.scala | 2 +- 6 files changed, 86 insertions(+), 82 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index ca4bf4709..142b2d159 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -376,12 +376,12 @@ object CometConf { .booleanConf .createWithDefault(false) - val COMET_CAST_STRING_TO_TIMESTAMP: ConfigEntry[Boolean] = conf( - "spark.comet.cast.stringToTimestamp") - .doc( - "Comet is not currently fully compatible with Spark when casting from String to Timestamp.") - .booleanConf - .createWithDefault(false) + val COMET_CAST_ALLOW_INCOMPATIBLE: ConfigEntry[Boolean] = + conf("spark.comet.cast.allowIncompatible") + .doc("Comet is not currently fully compatible with Spark for all cast operations.") + .booleanConf + // TODO change this to false and set this config explicitly in tests where needed + .createWithDefault(true) } diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 3a16cd47d..138e26cf2 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -25,7 +25,7 @@ Comet provides the following configuration settings. |--------|-------------|---------------| | spark.comet.ansi.enabled | Comet does not respect ANSI mode in most cases and by default will not accelerate queries when ansi mode is enabled. Enable this setting to test Comet's experimental support for ANSI mode. This should not be used in production. | false | | spark.comet.batchSize | The columnar batch size, i.e., the maximum number of rows that a batch can contain. | 8192 | -| spark.comet.cast.stringToTimestamp | Comet is not currently fully compatible with Spark when casting from String to Timestamp. | false | +| spark.comet.cast.allowIncompatible | Comet is not currently fully compatible with Spark for all cast operations. | true | | 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 | diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 04f14d9da..8f31561d6 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -1042,9 +1042,6 @@ object CometSparkSessionExtensions extends Logging { * The node with information (if any) attached */ def withInfo[T <: TreeNode[_]](node: T, info: String, exprs: T*): T = { - // scalastyle:off println - println("withInfo: " + info) - // scalastyle:on println // TODO maybe we could store the tags as `Set[String]` rather than newline-delimited strings // and avoid having to split and mkString val nodeInfo: Set[String] = node 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 9837d8967..402217c16 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -25,6 +25,17 @@ import org.apache.spark.sql.types.{DataType, DataTypes, DecimalType} import org.apache.comet.CometConf import org.apache.comet.CometSparkSessionExtensions.withInfo +sealed trait SupportLevel + +/** We support this feature with full compatibility with Spark */ +object Compatible extends SupportLevel + +/** We support this feature but results can be different from Spark */ +object Incompatible extends SupportLevel + +/** We do not support this feature */ +object Unsupported extends SupportLevel + object CometCast { def isSupported( @@ -32,41 +43,34 @@ object CometCast { fromType: DataType, toType: DataType, timeZoneId: Option[String], - evalMode: String): Boolean = { + evalMode: String): SupportLevel = { if (fromType == toType) { - return true + return Compatible } (fromType, toType) match { - - // TODO this is a temporary hack to allow casts that we either know are - // incompatible with Spark, or are just not well tested yet, just to avoid - // regressions in existing tests with this PR - - // BEGIN HACK case (dt: DataType, _) if dt.typeName == "timestamp_ntz" => toType match { case DataTypes.TimestampType | DataTypes.DateType | DataTypes.StringType => - true - case _ => false + Incompatible + case _ => + Unsupported } case (DataTypes.DoubleType, _: DecimalType) => - true + Incompatible case (DataTypes.TimestampType, DataTypes.LongType) => - true + Incompatible case (DataTypes.BinaryType | DataTypes.FloatType, DataTypes.StringType) => - true - case (_, DataTypes.BinaryType) => - true - // END HACK - + Incompatible + case (DataTypes.StringType, DataTypes.BinaryType) => + Incompatible case (_: DecimalType, _: DecimalType) => // TODO we need to file an issue for adding specific tests for casting // between decimal types with different precision and scale - true + Compatible case (DataTypes.StringType, _) => - canCastFromString(cast, toType) + canCastFromString(toType) case (_, DataTypes.StringType) => canCastToString(fromType) case (DataTypes.TimestampType, _) => @@ -83,99 +87,95 @@ object CometCast { canCastFromFloat(toType) case (DataTypes.DoubleType, _) => canCastFromDouble(toType) - case _ => false + case _ => Unsupported } } - private def canCastFromString(cast: Cast, toType: DataType): Boolean = { + private def canCastFromString(toType: DataType): SupportLevel = { toType match { case DataTypes.BooleanType => - true + Compatible case DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType => - true + Compatible case DataTypes.BinaryType => - true + Compatible case DataTypes.FloatType | DataTypes.DoubleType => // https://github.com/apache/datafusion-comet/issues/326 - false + Unsupported case _: DecimalType => // https://github.com/apache/datafusion-comet/issues/325 - false + Unsupported case DataTypes.DateType => // https://github.com/apache/datafusion-comet/issues/327 - false + Unsupported case DataTypes.TimestampType => - val enabled = CometConf.COMET_CAST_STRING_TO_TIMESTAMP.get() - if (!enabled) { - // https://github.com/apache/datafusion-comet/issues/328 - withInfo(cast, s"${CometConf.COMET_CAST_STRING_TO_TIMESTAMP.key} is disabled") - } - enabled + // https://github.com/apache/datafusion-comet/issues/328 + Incompatible case _ => - false + Unsupported } } - private def canCastToString(fromType: DataType): Boolean = { + private def canCastToString(fromType: DataType): SupportLevel = { fromType match { - case DataTypes.BooleanType => true + case DataTypes.BooleanType => Compatible case DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType => - true - case DataTypes.DateType => true - case DataTypes.TimestampType => true + Compatible + case DataTypes.DateType => Compatible + case DataTypes.TimestampType => Compatible case DataTypes.FloatType | DataTypes.DoubleType => // https://github.com/apache/datafusion-comet/issues/326 - false - case _ => false + Unsupported + case _ => Unsupported } } - private def canCastFromTimestamp(toType: DataType): Boolean = { + private def canCastFromTimestamp(toType: DataType): SupportLevel = { toType match { case DataTypes.BooleanType | DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType => // https://github.com/apache/datafusion-comet/issues/352 // this seems like an edge case that isn't important for us to support - false + Unsupported case DataTypes.LongType => // https://github.com/apache/datafusion-comet/issues/352 - false - case DataTypes.StringType => true - case DataTypes.DateType => true - case _ => false + Unsupported + case DataTypes.StringType => Compatible + case DataTypes.DateType => Compatible + case _ => Unsupported } } - private def canCastFromBoolean(toType: DataType) = toType match { + private def canCastFromBoolean(toType: DataType): SupportLevel = toType match { case DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | DataTypes.DoubleType => - true - case _ => false + Compatible + case _ => Unsupported } - private def canCastFromInt(toType: DataType) = toType match { + private def canCastFromInt(toType: DataType): SupportLevel = toType match { case DataTypes.BooleanType | DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | DataTypes.DoubleType | _: DecimalType => - true - case _ => false + Compatible + case _ => Unsupported } - private def canCastFromFloat(toType: DataType) = toType match { - case DataTypes.BooleanType | DataTypes.DoubleType => true - case _ => false + private def canCastFromFloat(toType: DataType): SupportLevel = toType match { + case DataTypes.BooleanType | DataTypes.DoubleType => Compatible + case _ => Unsupported } - private def canCastFromDouble(toType: DataType) = toType match { - case DataTypes.BooleanType | DataTypes.FloatType => true - case _ => false + private def canCastFromDouble(toType: DataType): SupportLevel = toType match { + case DataTypes.BooleanType | DataTypes.FloatType => Compatible + case _ => Unsupported } - private def canCastFromDecimal(toType: DataType) = toType match { - case DataTypes.FloatType | DataTypes.DoubleType => true - case _ => false + private def canCastFromDecimal(toType: DataType): SupportLevel = toType match { + case DataTypes.FloatType | DataTypes.DoubleType => Compatible + case _ => Unsupported } } 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 8da71f788..b496c7ce1 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -41,8 +41,9 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String +import org.apache.comet.CometConf import org.apache.comet.CometSparkSessionExtensions.{isCometOperatorEnabled, isCometScan, isSpark32, isSpark34Plus, withInfo} -import org.apache.comet.expressions.CometCast +import org.apache.comet.expressions.{CometCast, Compatible, Incompatible, Unsupported} import org.apache.comet.serde.ExprOuterClass.{AggExpr, DataType => ProtoDataType, Expr, ScalarFunc} import org.apache.comet.serde.ExprOuterClass.DataType.{DataTypeInfo, DecimalInfo, ListInfo, MapInfo, StructInfo} import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, JoinType, Operator} @@ -585,14 +586,20 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { // Spark 3.4+ has EvalMode enum with values LEGACY, ANSI, and TRY evalMode.toString } - if (CometCast.isSupported(cast, child.dataType, dt, timeZoneId, evalModeStr)) { - castToProto(timeZoneId, dt, childExpr, evalModeStr) - } else { - withInfo( - expr, - s"Unsupported cast from ${child.dataType} to $dt " + - s"with timezone $timeZoneId and evalMode $evalModeStr") - None + val castSupport = + CometCast.isSupported(cast, child.dataType, dt, timeZoneId, evalModeStr) + castSupport match { + case Compatible => + castToProto(timeZoneId, dt, childExpr, evalModeStr) + case Incompatible if CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.get() => + logWarning(s"Calling incompatible CAST expression: $cast") + castToProto(timeZoneId, dt, childExpr, evalModeStr) + case Unsupported => + withInfo( + expr, + s"Unsupported cast from ${child.dataType} to $dt " + + s"with timezone $timeZoneId and evalMode $evalModeStr") + None } } else { withInfo(expr, child) diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index fd108c4c3..f4f94fbed 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -551,7 +551,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { ignore("cast StringType to TimestampType") { // https://github.com/apache/datafusion-comet/issues/328 - withSQLConf((CometConf.COMET_CAST_STRING_TO_TIMESTAMP.key, "true")) { + withSQLConf((CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key, "true")) { val values = Seq("2020-01-01T12:34:56.123456", "T2") ++ generateStrings(timestampPattern, 8) castTest(values.toDF("a"), DataTypes.TimestampType) } From 5be20c621be8e7a87e2363b8d500b89aa8140d68 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 10:17:01 -0600 Subject: [PATCH 10/38] remove unused imports --- .../main/scala/org/apache/comet/expressions/CometCast.scala | 3 --- 1 file changed, 3 deletions(-) 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 402217c16..2f5a6bfef 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -22,9 +22,6 @@ package org.apache.comet.expressions import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.types.{DataType, DataTypes, DecimalType} -import org.apache.comet.CometConf -import org.apache.comet.CometSparkSessionExtensions.withInfo - sealed trait SupportLevel /** We support this feature with full compatibility with Spark */ From 23847cd6838872eddcb70b2c19ed06be6cf078a0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 10:49:12 -0600 Subject: [PATCH 11/38] revert some changes that are no longer needed --- .../comet/CometSparkSessionExtensions.scala | 21 ++++++------------ .../apache/comet/CometExpressionSuite.scala | 22 ------------------- 2 files changed, 7 insertions(+), 36 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 8f31561d6..1e78013c7 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -1042,21 +1042,14 @@ object CometSparkSessionExtensions extends Logging { * The node with information (if any) attached */ def withInfo[T <: TreeNode[_]](node: T, info: String, exprs: T*): T = { - // TODO maybe we could store the tags as `Set[String]` rather than newline-delimited strings - // and avoid having to split and mkString - val nodeInfo: Set[String] = node - .getTagValue(CometExplainInfo.EXTENSION_INFO) - .map(_.split('\n').toSet) - .getOrElse(Set.empty[String]) - val exprInfo: Set[String] = exprs - .flatMap(e => e.getTagValue(CometExplainInfo.EXTENSION_INFO).map(_.split('\n'))) + val exprInfo = exprs + .flatMap { e => Seq(e.getTagValue(CometExplainInfo.EXTENSION_INFO)) } .flatten - .toSet - val currentInfo = nodeInfo ++ exprInfo - if (info != null && info.nonEmpty && currentInfo.nonEmpty) { - node.setTagValue(CometExplainInfo.EXTENSION_INFO, (currentInfo + info).mkString("\n")) - } else if (currentInfo.nonEmpty) { - node.setTagValue(CometExplainInfo.EXTENSION_INFO, currentInfo.mkString("\n")) + .mkString("\n") + if (info != null && info.nonEmpty && exprInfo.nonEmpty) { + node.setTagValue(CometExplainInfo.EXTENSION_INFO, Seq(exprInfo, info).mkString("\n")) + } else if (exprInfo.nonEmpty) { + node.setTagValue(CometExplainInfo.EXTENSION_INFO, exprInfo) } else if (info != null && info.nonEmpty) { node.setTagValue(CometExplainInfo.EXTENSION_INFO, info) } diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index a63f459ad..3683c8d44 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -21,7 +21,6 @@ package org.apache.comet import org.apache.hadoop.fs.Path import org.apache.spark.sql.{CometTestBase, DataFrame, Row} -import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.expr import org.apache.spark.sql.internal.SQLConf @@ -1428,25 +1427,4 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - test("tag nodes with multiple info") { - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { - val project = spark - .sql("SELECT CAST(1 as string)") - .queryExecution - .executedPlan - .asInstanceOf[ProjectExec] - assert(project.getTagValue(CometExplainInfo.EXTENSION_INFO).isEmpty) - CometSparkSessionExtensions.withInfo(project.expressions.head, "expr info 1") - CometSparkSessionExtensions.withInfo(project, "node info 1", project.expressions: _*) - assert( - project.getTagValue(CometExplainInfo.EXTENSION_INFO).get == "expr info 1\nnode info 1") - CometSparkSessionExtensions.withInfo(project, "node info 2", project.expressions: _*) - assert( - project - .getTagValue(CometExplainInfo.EXTENSION_INFO) - .get == "expr info 1\nnode info 1\nnode info 2") - } - - } - } From 4bf6c16563631a80d81aef7b9b9c7eed4d505140 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 11:00:20 -0600 Subject: [PATCH 12/38] save progress --- .../scala/org/apache/comet/CometConf.scala | 4 +++- .../apache/comet/expressions/CometCast.scala | 20 ++++++++++--------- .../org/apache/comet/CometCastSuite.scala | 15 ++++++++------ 3 files changed, 23 insertions(+), 16 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 142b2d159..5d86d3c5d 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -378,7 +378,9 @@ object CometConf { val COMET_CAST_ALLOW_INCOMPATIBLE: ConfigEntry[Boolean] = conf("spark.comet.cast.allowIncompatible") - .doc("Comet is not currently fully compatible with Spark for all cast operations.") + .doc( + "Comet is not currently fully compatible with Spark for all cast operations. Set this config " + + "to true to allow them anyway. See compatibility guide for more information.") .booleanConf // TODO change this to false and set this config explicitly in tests where needed .createWithDefault(true) 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 9274f1f18..f66801732 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -62,16 +62,14 @@ object CometCast { Incompatible case (DataTypes.TimestampType, DataTypes.LongType) => Incompatible - case (DataTypes.BinaryType | DataTypes.FloatType, DataTypes.StringType) => - Incompatible - case (DataTypes.StringType, DataTypes.BinaryType) => + case (DataTypes.BinaryType, DataTypes.StringType) => Incompatible case (_: DecimalType, _: DecimalType) => // TODO we need to file an issue for adding specific tests for casting // between decimal types with different precision and scale - Compatible + Incompatible case (DataTypes.StringType, _) => - canCastFromString(toType, timeZoneId) + canCastFromString(toType, timeZoneId, evalMode) case (_, DataTypes.StringType) => canCastToString(fromType) case (DataTypes.TimestampType, _) => @@ -92,7 +90,10 @@ object CometCast { } } - private def canCastFromString(toType: DataType, timeZoneId: Option[String]): SupportLevel = { + private def canCastFromString( + toType: DataType, + timeZoneId: Option[String], + evalMode: String): SupportLevel = { toType match { case DataTypes.BooleanType => Compatible @@ -111,8 +112,9 @@ object CometCast { // https://github.com/apache/datafusion-comet/issues/327 Unsupported case DataTypes.TimestampType if !timeZoneId.contains("UTC") => - UnsupportedWithReason( - s"Unsupported timezone $timeZoneId for cast from string to timestamp") + UnsupportedWithReason(s"Unsupported timezone $timeZoneId") + case DataTypes.TimestampType if evalMode == "ANSI" => + UnsupportedWithReason(s"ANSI mode not supported") case DataTypes.TimestampType => // https://github.com/apache/datafusion-comet/issues/328 Incompatible @@ -131,7 +133,7 @@ object CometCast { case DataTypes.TimestampType => Compatible case DataTypes.FloatType | DataTypes.DoubleType => // https://github.com/apache/datafusion-comet/issues/326 - Unsupported + Incompatible case _ => Unsupported } } diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 1481d2f2e..ad7619285 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -65,8 +65,6 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } else if (!testExists) { fail(s"Missing test: $expectedTestName") } - } else if (testExists) { - fail(s"Found test for cast that Spark does not support: $expectedTestName") } } } @@ -543,10 +541,15 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("cast StringType to TimestampType disabled by default") { val values = Seq("2020-01-01T12:34:56.123456", "T2").toDF("a") - castFallbackTest( - values.toDF("a"), - DataTypes.TimestampType, - "spark.comet.cast.stringToTimestamp is disabled") + castFallbackTest(values.toDF("a"), DataTypes.TimestampType, "Unsupported timezone") + } + + ignore("cast StringType to TimestampType (fuzz test)") { + // https://github.com/apache/datafusion-comet/issues/328 + withSQLConf((CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key, "true")) { + val values = Seq("2020-01-01T12:34:56.123456", "T2") ++ generateStrings(timestampPattern, 8) + castTest(values.toDF("a"), DataTypes.TimestampType) + } } test("cast StringType to TimestampType") { From ed0791395cb3a93374f3a798ef4cb99aaf52eb31 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 11:27:04 -0600 Subject: [PATCH 13/38] checkpoint --- common/src/main/scala/org/apache/comet/CometConf.scala | 5 +++-- .../scala/org/apache/comet/expressions/CometCast.scala | 4 ++-- .../src/test/scala/org/apache/comet/CometCastSuite.scala | 8 ++++---- .../test/scala/org/apache/spark/sql/CometTestBase.scala | 3 +++ 4 files changed, 12 insertions(+), 8 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 5d86d3c5d..14360ee66 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -379,8 +379,9 @@ object CometConf { val COMET_CAST_ALLOW_INCOMPATIBLE: ConfigEntry[Boolean] = conf("spark.comet.cast.allowIncompatible") .doc( - "Comet is not currently fully compatible with Spark for all cast operations. Set this config " + - "to true to allow them anyway. See compatibility guide for more information.") + "Comet is not currently fully compatible with Spark for all cast operations. " + + "Set this config to true to allow them anyway. See compatibility guide " + + "for more information.") .booleanConf // TODO change this to false and set this config explicitly in tests where needed .createWithDefault(true) 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 f66801732..77ee4fc41 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -117,7 +117,7 @@ object CometCast { UnsupportedWithReason(s"ANSI mode not supported") case DataTypes.TimestampType => // https://github.com/apache/datafusion-comet/issues/328 - Incompatible + Compatible case _ => Unsupported } @@ -147,7 +147,7 @@ object CometCast { Unsupported case DataTypes.LongType => // https://github.com/apache/datafusion-comet/issues/352 - Unsupported + Compatible case DataTypes.StringType => Compatible case DataTypes.DateType => Compatible case _ => Unsupported diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index ad7619285..a62b9be0a 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -19,10 +19,10 @@ package org.apache.comet -import java.io.File +import org.apache.comet.expressions.{CometCast, Compatible} +import java.io.File import scala.util.Random - import org.apache.spark.sql.{CometTestBase, DataFrame, SaveMode} import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -167,7 +167,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateBytes(), DataTypes.BinaryType) } - ignore("cast ByteType to TimestampType") { + test("cast ByteType to TimestampType") { // input: -1, expected: 1969-12-31 15:59:59.0, actual: 1969-12-31 15:59:59.999999 castTest(generateBytes(), DataTypes.TimestampType) } @@ -665,7 +665,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateTimestamps(), DataTypes.IntegerType) } - ignore("cast TimestampType to LongType") { + test("cast TimestampType to LongType") { // https://github.com/apache/datafusion-comet/issues/352 // input: 2023-12-31 17:00:00.0, expected: 1.70407078E9, actual: 1.70407082E15] castTest(generateTimestamps(), DataTypes.LongType) 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 8fda13617..e21c7b52a 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -144,10 +144,13 @@ abstract class CometTestBase withSQLConf(CometConf.COMET_ENABLED.key -> "false") { val dfSpark = Dataset.ofRows(spark, df.logicalPlan) expected = dfSpark.collect() + dfSpark.show() sparkPlan = dfSpark.queryExecution.executedPlan } val dfComet = Dataset.ofRows(spark, df.logicalPlan) checkAnswer(dfComet, expected) + dfComet.show() + println(dfComet.queryExecution.executedPlan) (sparkPlan, dfComet.queryExecution.executedPlan) } From 5255d6cdf07639ac8a1d7f25baf54a3cf6cd33ac Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 11:49:05 -0600 Subject: [PATCH 14/38] save --- spark/src/test/scala/org/apache/comet/CometCastSuite.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index a62b9be0a..2fb8c1883 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -167,7 +167,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateBytes(), DataTypes.BinaryType) } - test("cast ByteType to TimestampType") { + ignore("cast ByteType to TimestampType") { // input: -1, expected: 1969-12-31 15:59:59.0, actual: 1969-12-31 15:59:59.999999 castTest(generateBytes(), DataTypes.TimestampType) } @@ -666,8 +666,6 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } test("cast TimestampType to LongType") { - // https://github.com/apache/datafusion-comet/issues/352 - // input: 2023-12-31 17:00:00.0, expected: 1.70407078E9, actual: 1.70407082E15] castTest(generateTimestamps(), DataTypes.LongType) } From 0c8da561ca67ac7adc610dabb3d0a7fe3b03f96c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 12:11:42 -0600 Subject: [PATCH 15/38] code cleanup --- .../apache/comet/expressions/CometCast.scala | 52 +++++++++---------- .../apache/comet/serde/QueryPlanSerde.scala | 19 +++---- 2 files changed, 32 insertions(+), 39 deletions(-) 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 77ee4fc41..7ce74c87d 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -29,13 +29,10 @@ sealed trait SupportLevel object Compatible extends SupportLevel /** We support this feature but results can be different from Spark */ -object Incompatible extends SupportLevel +case class Incompatible(reason: Option[String] = None) extends SupportLevel /** We do not support this feature */ -object Unsupported extends SupportLevel - -/** We do not support this feature and we explain why */ -case class UnsupportedWithReason(reason: String) extends SupportLevel +case class Unsupported(reason: Option[String] = None) extends SupportLevel object CometCast { @@ -54,20 +51,18 @@ object CometCast { case (dt: DataType, _) if dt.typeName == "timestamp_ntz" => toType match { case DataTypes.TimestampType | DataTypes.DateType | DataTypes.StringType => - Incompatible + Incompatible() case _ => - Unsupported + Unsupported() } - case (DataTypes.DoubleType, _: DecimalType) => - Incompatible case (DataTypes.TimestampType, DataTypes.LongType) => - Incompatible + Incompatible() case (DataTypes.BinaryType, DataTypes.StringType) => - Incompatible + Incompatible() case (_: DecimalType, _: DecimalType) => // TODO we need to file an issue for adding specific tests for casting // between decimal types with different precision and scale - Incompatible + Incompatible() case (DataTypes.StringType, _) => canCastFromString(toType, timeZoneId, evalMode) case (_, DataTypes.StringType) => @@ -86,7 +81,7 @@ object CometCast { canCastFromFloat(toType) case (DataTypes.DoubleType, _) => canCastFromDouble(toType) - case _ => Unsupported + case _ => Unsupported() } } @@ -104,22 +99,22 @@ object CometCast { Compatible case DataTypes.FloatType | DataTypes.DoubleType => // https://github.com/apache/datafusion-comet/issues/326 - Unsupported + Unsupported() case _: DecimalType => // https://github.com/apache/datafusion-comet/issues/325 - Unsupported + Unsupported() case DataTypes.DateType => // https://github.com/apache/datafusion-comet/issues/327 - Unsupported + Unsupported() case DataTypes.TimestampType if !timeZoneId.contains("UTC") => - UnsupportedWithReason(s"Unsupported timezone $timeZoneId") + Unsupported(Some(s"Unsupported timezone $timeZoneId")) case DataTypes.TimestampType if evalMode == "ANSI" => - UnsupportedWithReason(s"ANSI mode not supported") + Unsupported(Some(s"ANSI mode not supported")) case DataTypes.TimestampType => // https://github.com/apache/datafusion-comet/issues/328 Compatible case _ => - Unsupported + Unsupported() } } @@ -133,8 +128,8 @@ object CometCast { case DataTypes.TimestampType => Compatible case DataTypes.FloatType | DataTypes.DoubleType => // https://github.com/apache/datafusion-comet/issues/326 - Incompatible - case _ => Unsupported + Incompatible() + case _ => Unsupported() } } @@ -144,13 +139,13 @@ object CometCast { DataTypes.IntegerType => // https://github.com/apache/datafusion-comet/issues/352 // this seems like an edge case that isn't important for us to support - Unsupported + Unsupported() case DataTypes.LongType => // https://github.com/apache/datafusion-comet/issues/352 Compatible case DataTypes.StringType => Compatible case DataTypes.DateType => Compatible - case _ => Unsupported + case _ => Unsupported() } } @@ -158,7 +153,7 @@ object CometCast { case DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | DataTypes.DoubleType => Compatible - case _ => Unsupported + case _ => Unsupported() } private def canCastFromInt(toType: DataType): SupportLevel = toType match { @@ -166,22 +161,23 @@ object CometCast { DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | DataTypes.DoubleType | _: DecimalType => Compatible - case _ => Unsupported + case _ => Unsupported() } private def canCastFromFloat(toType: DataType): SupportLevel = toType match { case DataTypes.BooleanType | DataTypes.DoubleType => Compatible - case _ => Unsupported + case _ => Unsupported() } private def canCastFromDouble(toType: DataType): SupportLevel = toType match { case DataTypes.BooleanType | DataTypes.FloatType => Compatible - case _ => Unsupported + case _: DecimalType => Incompatible() + case _ => Unsupported() } private def canCastFromDecimal(toType: DataType): SupportLevel = toType match { case DataTypes.FloatType | DataTypes.DoubleType => Compatible - case _ => Unsupported + case _ => Unsupported() } } 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 e3bbd6de7..12ed31ff4 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -43,7 +43,7 @@ import org.apache.spark.unsafe.types.UTF8String import org.apache.comet.CometConf import org.apache.comet.CometSparkSessionExtensions.{isCometOperatorEnabled, isCometScan, isSpark32, isSpark34Plus, withInfo} -import org.apache.comet.expressions.{CometCast, Compatible, Incompatible, Unsupported, UnsupportedWithReason} +import org.apache.comet.expressions.{CometCast, Compatible, Incompatible, Unsupported} import org.apache.comet.serde.ExprOuterClass.{AggExpr, DataType => ProtoDataType, Expr, ScalarFunc} import org.apache.comet.serde.ExprOuterClass.DataType.{DataTypeInfo, DecimalInfo, ListInfo, MapInfo, StructInfo} import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, JoinType, Operator} @@ -591,20 +591,17 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { castSupport match { case Compatible => castToProto(timeZoneId, dt, childExpr, evalModeStr) - case Incompatible if CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.get() => - logWarning(s"Calling incompatible CAST expression: $cast") + case Incompatible(reason) if CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.get() => + logWarning(s"Calling incompatible CAST expression: $cast" + + reason.map(str => s" ($str)").getOrElse("")) castToProto(timeZoneId, dt, childExpr, evalModeStr) - case UnsupportedWithReason(reason) => + case Unsupported(reason) => withInfo( expr, s"Unsupported cast from ${child.dataType} to $dt " + - s"with timezone $timeZoneId and evalMode $evalModeStr: $reason") - None - case Unsupported => - withInfo( - expr, - s"Unsupported cast from ${child.dataType} to $dt " + - s"with timezone $timeZoneId and evalMode $evalModeStr") + s"with timezone $timeZoneId and evalMode $evalModeStr" + + reason.map(str => s" ($str)").getOrElse("") + ) None } } else { From 22c6394f968e7829d00ef5d8c6699a30be6315c0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 12:14:14 -0600 Subject: [PATCH 16/38] fix bug --- .../apache/comet/serde/QueryPlanSerde.scala | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) 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 12ed31ff4..3045c667a 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -591,10 +591,22 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { castSupport match { case Compatible => castToProto(timeZoneId, dt, childExpr, evalModeStr) - case Incompatible(reason) if CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.get() => - logWarning(s"Calling incompatible CAST expression: $cast" + - reason.map(str => s" ($str)").getOrElse("")) - castToProto(timeZoneId, dt, childExpr, evalModeStr) + case Incompatible(reason) => + if (CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.get()){ + logWarning(s"Calling incompatible CAST expression: $cast" + + reason.map(str => s" ($str)").getOrElse("")) + castToProto(timeZoneId, dt, childExpr, evalModeStr) + } else { + withInfo( + expr, + s"Comet does not guarantee correct results for cast " + + s"from ${child.dataType} to $dt " + + s"with timezone $timeZoneId and evalMode $evalModeStr" + + reason.map(str => s" ($str)").getOrElse("") + "." + + s"To enable all incompatible casts, set " + + s"${CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key}=true" + ) + } case Unsupported(reason) => withInfo( expr, From 7c39b0595fde84201d423c355c4fbd98572b9fe3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 12:15:36 -0600 Subject: [PATCH 17/38] fix bug --- spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala | 1 + 1 file changed, 1 insertion(+) 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 3045c667a..880f8d02b 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -606,6 +606,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { s"To enable all incompatible casts, set " + s"${CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key}=true" ) + None } case Unsupported(reason) => withInfo( From 787d17b121753867a2f6770ef785d537587d5215 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 12:20:36 -0600 Subject: [PATCH 18/38] improve reporting --- .../apache/comet/serde/QueryPlanSerde.scala | 20 ++++++++++--------- .../org/apache/comet/CometCastSuite.scala | 2 -- .../org/apache/spark/sql/CometTestBase.scala | 3 --- 3 files changed, 11 insertions(+), 14 deletions(-) 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 880f8d02b..5f7faf4ef 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -588,23 +588,25 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { } val castSupport = CometCast.isSupported(cast, child.dataType, dt, timeZoneId, evalModeStr) + + def getIncompatMessage(reason: Option[String]) = + s"Comet does not guarantee correct results for cast " + + s"from ${child.dataType} to $dt " + + s"with timezone $timeZoneId and evalMode $evalModeStr" + + reason.map(str => s" ($str)").getOrElse("") + castSupport match { case Compatible => castToProto(timeZoneId, dt, childExpr, evalModeStr) case Incompatible(reason) => - if (CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.get()){ - logWarning(s"Calling incompatible CAST expression: $cast" + - reason.map(str => s" ($str)").getOrElse("")) + if (CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.get()) { + logWarning(getIncompatMessage(reason)) castToProto(timeZoneId, dt, childExpr, evalModeStr) } else { withInfo( expr, - s"Comet does not guarantee correct results for cast " + - s"from ${child.dataType} to $dt " + - s"with timezone $timeZoneId and evalMode $evalModeStr" + - reason.map(str => s" ($str)").getOrElse("") + "." + - s"To enable all incompatible casts, set " + - s"${CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key}=true" + s"${getIncompatMessage(reason)}. To enable all incompatible casts, set " + + s"${CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key}=true" ) None } diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 2fb8c1883..2a32dd9e9 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -19,8 +19,6 @@ package org.apache.comet -import org.apache.comet.expressions.{CometCast, Compatible} - import java.io.File import scala.util.Random import org.apache.spark.sql.{CometTestBase, DataFrame, SaveMode} 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 e21c7b52a..8fda13617 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -144,13 +144,10 @@ abstract class CometTestBase withSQLConf(CometConf.COMET_ENABLED.key -> "false") { val dfSpark = Dataset.ofRows(spark, df.logicalPlan) expected = dfSpark.collect() - dfSpark.show() sparkPlan = dfSpark.queryExecution.executedPlan } val dfComet = Dataset.ofRows(spark, df.logicalPlan) checkAnswer(dfComet, expected) - dfComet.show() - println(dfComet.queryExecution.executedPlan) (sparkPlan, dfComet.queryExecution.executedPlan) } From 1b833e20d0529205b43189f5c09fdd7b6c68d5e9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 13:45:24 -0600 Subject: [PATCH 19/38] add documentation --- docs/source/user-guide/compatibility.md | 138 +++++++++++++++++- .../apache/comet/expressions/CometCast.scala | 41 +++--- .../apache/comet/serde/QueryPlanSerde.scala | 7 +- .../org/apache/comet/CometCastSuite.scala | 60 ++++++-- 4 files changed, 196 insertions(+), 50 deletions(-) diff --git a/docs/source/user-guide/compatibility.md b/docs/source/user-guide/compatibility.md index b4b4c92eb..3cb6bb4d8 100644 --- a/docs/source/user-guide/compatibility.md +++ b/docs/source/user-guide/compatibility.md @@ -34,13 +34,135 @@ There is an [epic](https://github.com/apache/datafusion-comet/issues/313) where ## Cast -Comet currently delegates to Apache DataFusion for most cast operations, and this means that the behavior is not -guaranteed to be consistent with Spark. +Cast operations in Comet fall into three levels of support: -There is an [epic](https://github.com/apache/datafusion-comet/issues/286) where we are tracking the work to implement Spark-compatible cast expressions. +- Compatible: The results match Apache Spark +- Incompatible: The results may match with Apache Spark for some inputs, but there are known issues where some + inputs will result in incorrect results or exceptions. The query stage will fall back to Spark by default. Setting + `spark.comet.cast.allowIncompatible=true` will allow all incompatible casts to run natively in Comet, but this is not + recommended for production use. +- Unsupported: Comet does not provide a native version of this cast expression and the query stage will fall back to + Spark. -### Cast from String to Timestamp - -Casting from String to Timestamp is disabled by default due to incompatibilities with Spark, including timezone -issues, and can be enabled by setting `spark.comet.castStringToTimestamp=true`. See the -[tracking issue](https://github.com/apache/datafusion-comet/issues/328) for more information. +| From Type | To Type | Compatible? | Notes | +| --------- | --------- | ------------ | ----------------------------------- | +| boolean | boolean | Compatible | | +| boolean | byte | Compatible | | +| boolean | short | Compatible | | +| boolean | integer | Compatible | | +| boolean | long | Compatible | | +| boolean | float | Compatible | | +| boolean | double | Compatible | | +| boolean | decimal | Unsupported | | +| boolean | string | Compatible | | +| boolean | timestamp | Unsupported | | +| byte | boolean | Compatible | | +| byte | byte | Compatible | | +| byte | short | Compatible | | +| byte | integer | Compatible | | +| byte | long | Compatible | | +| byte | float | Compatible | | +| byte | double | Compatible | | +| byte | decimal | Compatible | | +| byte | string | Compatible | | +| byte | binary | Unsupported | | +| byte | timestamp | Unsupported | | +| short | boolean | Compatible | | +| short | byte | Compatible | | +| short | short | Compatible | | +| short | integer | Compatible | | +| short | long | Compatible | | +| short | float | Compatible | | +| short | double | Compatible | | +| short | decimal | Compatible | | +| short | string | Compatible | | +| short | binary | Unsupported | | +| short | timestamp | Unsupported | | +| integer | boolean | Compatible | | +| integer | byte | Compatible | | +| integer | short | Compatible | | +| integer | integer | Compatible | | +| integer | long | Compatible | | +| integer | float | Compatible | | +| integer | double | Compatible | | +| integer | decimal | Compatible | | +| integer | string | Compatible | | +| integer | binary | Unsupported | | +| integer | timestamp | Unsupported | | +| long | boolean | Compatible | | +| long | byte | Compatible | | +| long | short | Compatible | | +| long | integer | Compatible | | +| long | long | Compatible | | +| long | float | Compatible | | +| long | double | Compatible | | +| long | decimal | Compatible | | +| long | string | Compatible | | +| long | binary | Unsupported | | +| long | timestamp | Unsupported | | +| float | boolean | Compatible | | +| float | byte | Unsupported | | +| float | short | Unsupported | | +| float | integer | Unsupported | | +| float | long | Unsupported | | +| float | float | Compatible | | +| float | double | Compatible | | +| float | decimal | Unsupported | | +| float | string | Incompatible | | +| float | timestamp | Unsupported | | +| double | boolean | Compatible | | +| double | byte | Unsupported | | +| double | short | Unsupported | | +| double | integer | Unsupported | | +| double | long | Unsupported | | +| double | float | Compatible | | +| double | double | Compatible | | +| double | decimal | Incompatible | | +| double | string | Incompatible | | +| double | timestamp | Unsupported | | +| decimal | boolean | Unsupported | | +| decimal | byte | Unsupported | | +| decimal | short | Unsupported | | +| decimal | integer | Unsupported | | +| decimal | long | Unsupported | | +| decimal | float | Compatible | | +| decimal | double | Compatible | | +| decimal | decimal | Compatible | | +| decimal | string | Unsupported | | +| decimal | timestamp | Unsupported | | +| string | boolean | Compatible | | +| string | byte | Compatible | | +| string | short | Compatible | | +| string | integer | Compatible | | +| string | long | Compatible | | +| string | float | Unsupported | | +| string | double | Unsupported | | +| string | decimal | Unsupported | | +| string | string | Compatible | | +| string | binary | Compatible | | +| string | date | Unsupported | | +| string | timestamp | Incompatible | Not all valid formats are supported | +| binary | string | Incompatible | | +| binary | binary | Compatible | | +| date | boolean | Unsupported | | +| date | byte | Unsupported | | +| date | short | Unsupported | | +| date | integer | Unsupported | | +| date | long | Unsupported | | +| date | float | Unsupported | | +| date | double | Unsupported | | +| date | decimal | Unsupported | | +| date | string | Compatible | | +| date | date | Compatible | | +| date | timestamp | Unsupported | | +| timestamp | boolean | Unsupported | | +| timestamp | byte | Unsupported | | +| timestamp | short | Unsupported | | +| timestamp | integer | Unsupported | | +| timestamp | long | Compatible | | +| timestamp | float | Unsupported | | +| timestamp | double | Unsupported | | +| timestamp | decimal | Unsupported | | +| timestamp | string | Compatible | | +| timestamp | date | Compatible | | +| timestamp | timestamp | Compatible | | 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 7ce74c87d..16782a4bd 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -32,12 +32,11 @@ object Compatible extends SupportLevel case class Incompatible(reason: Option[String] = None) extends SupportLevel /** We do not support this feature */ -case class Unsupported(reason: Option[String] = None) extends SupportLevel +object Unsupported extends SupportLevel object CometCast { def isSupported( - cast: Cast, fromType: DataType, toType: DataType, timeZoneId: Option[String], @@ -53,10 +52,8 @@ object CometCast { case DataTypes.TimestampType | DataTypes.DateType | DataTypes.StringType => Incompatible() case _ => - Unsupported() + Unsupported } - case (DataTypes.TimestampType, DataTypes.LongType) => - Incompatible() case (DataTypes.BinaryType, DataTypes.StringType) => Incompatible() case (_: DecimalType, _: DecimalType) => @@ -81,7 +78,7 @@ object CometCast { canCastFromFloat(toType) case (DataTypes.DoubleType, _) => canCastFromDouble(toType) - case _ => Unsupported() + case _ => Unsupported } } @@ -99,22 +96,22 @@ object CometCast { Compatible case DataTypes.FloatType | DataTypes.DoubleType => // https://github.com/apache/datafusion-comet/issues/326 - Unsupported() + Unsupported case _: DecimalType => // https://github.com/apache/datafusion-comet/issues/325 - Unsupported() + Unsupported case DataTypes.DateType => // https://github.com/apache/datafusion-comet/issues/327 - Unsupported() - case DataTypes.TimestampType if !timeZoneId.contains("UTC") => - Unsupported(Some(s"Unsupported timezone $timeZoneId")) + Unsupported + 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" => - Unsupported(Some(s"ANSI mode not supported")) + Incompatible(Some(s"ANSI mode not supported")) case DataTypes.TimestampType => // https://github.com/apache/datafusion-comet/issues/328 - Compatible + Incompatible(Some("Not all valid formats are supported")) case _ => - Unsupported() + Unsupported } } @@ -129,7 +126,7 @@ object CometCast { case DataTypes.FloatType | DataTypes.DoubleType => // https://github.com/apache/datafusion-comet/issues/326 Incompatible() - case _ => Unsupported() + case _ => Unsupported } } @@ -139,13 +136,13 @@ object CometCast { DataTypes.IntegerType => // https://github.com/apache/datafusion-comet/issues/352 // this seems like an edge case that isn't important for us to support - Unsupported() + Unsupported case DataTypes.LongType => // https://github.com/apache/datafusion-comet/issues/352 Compatible case DataTypes.StringType => Compatible case DataTypes.DateType => Compatible - case _ => Unsupported() + case _ => Unsupported } } @@ -153,7 +150,7 @@ object CometCast { case DataTypes.ByteType | DataTypes.ShortType | DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | DataTypes.DoubleType => Compatible - case _ => Unsupported() + case _ => Unsupported } private def canCastFromInt(toType: DataType): SupportLevel = toType match { @@ -161,23 +158,23 @@ object CometCast { DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | DataTypes.DoubleType | _: DecimalType => Compatible - case _ => Unsupported() + case _ => Unsupported } private def canCastFromFloat(toType: DataType): SupportLevel = toType match { case DataTypes.BooleanType | DataTypes.DoubleType => Compatible - case _ => Unsupported() + case _ => Unsupported } private def canCastFromDouble(toType: DataType): SupportLevel = toType match { case DataTypes.BooleanType | DataTypes.FloatType => Compatible case _: DecimalType => Incompatible() - case _ => Unsupported() + case _ => Unsupported } private def canCastFromDecimal(toType: DataType): SupportLevel = toType match { case DataTypes.FloatType | DataTypes.DoubleType => Compatible - case _ => Unsupported() + case _ => Unsupported } } 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 5f7faf4ef..f0406234b 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -587,7 +587,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { evalMode.toString } val castSupport = - CometCast.isSupported(cast, child.dataType, dt, timeZoneId, evalModeStr) + CometCast.isSupported(child.dataType, dt, timeZoneId, evalModeStr) def getIncompatMessage(reason: Option[String]) = s"Comet does not guarantee correct results for cast " + @@ -610,12 +610,11 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { ) None } - case Unsupported(reason) => + case Unsupported => withInfo( expr, s"Unsupported cast from ${child.dataType} to $dt " + - s"with timezone $timeZoneId and evalMode $evalModeStr" + - reason.map(str => s" ($str)").getOrElse("") + s"with timezone $timeZoneId and evalMode $evalModeStr" ) None } diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 2a32dd9e9..f9c426125 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -19,6 +19,8 @@ package org.apache.comet +import org.apache.comet.expressions.{CometCast, Compatible, Incompatible, Unsupported} + import java.io.File import scala.util.Random import org.apache.spark.sql.{CometTestBase, DataFrame, SaveMode} @@ -47,6 +49,23 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { private val datePattern = "0123456789/" + whitespaceChars private val timestampPattern = "0123456789/:T" + whitespaceChars + // make sure we have tests for all combinations of our supported types + val supportedTypes = + Seq( + DataTypes.BooleanType, + DataTypes.ByteType, + DataTypes.ShortType, + DataTypes.IntegerType, + DataTypes.LongType, + DataTypes.FloatType, + DataTypes.DoubleType, + DataTypes.createDecimalType(10, 2), + DataTypes.StringType, + DataTypes.BinaryType, + DataTypes.DateType, + DataTypes.TimestampType) + // TODO add DataTypes.TimestampNTZType for Spark 3.4 and later + test("all valid cast combinations covered") { val names = testNames @@ -68,25 +87,34 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - // make sure we have tests for all combinations of our supported types - val supportedTypes = - Seq( - DataTypes.BooleanType, - DataTypes.ByteType, - DataTypes.ShortType, - DataTypes.IntegerType, - DataTypes.LongType, - DataTypes.FloatType, - DataTypes.DoubleType, - DataTypes.createDecimalType(10, 2), - DataTypes.StringType, - DataTypes.BinaryType, - DataTypes.DateType, - DataTypes.TimestampType) - // TODO add DataTypes.TimestampNTZType for Spark 3.4 and later assertTestsExist(supportedTypes, supportedTypes) } + test("generate cast documentation") { + // scalastyle:on println + println(s"| From Type | To Type | Compatible? | Notes |") + println(s"|-|-|-|-|") + for (fromType <- supportedTypes) { + for (toType <- supportedTypes) { + if (Cast.canCast(fromType, toType)) { + val fromTypeName = fromType.typeName.replace("(10,2)", "") + val toTypeName = toType.typeName.replace("(10,2)", "") + CometCast.isSupported(fromType, toType, None, "LEGACY") match { + case Compatible => + println(s"| $fromTypeName | $toTypeName | Compatible | |") + case Incompatible(Some(reason)) => + println(s"| $fromTypeName | $toTypeName | Incompatible | $reason |") + case Incompatible(None) => + println(s"| $fromTypeName | $toTypeName | Incompatible | |") + case Unsupported => + println(s"| $fromTypeName | $toTypeName | Unsupported | |") + } + } + } + } + // scalastyle:off println + } + // CAST from BooleanType test("cast BooleanType to ByteType") { From 7ad2a43f023ad6119c9b9194a1ba9e0580305b36 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 13:53:27 -0600 Subject: [PATCH 20/38] improve docs, fix code format --- docs/source/user-guide/compatibility.md | 11 +++++++---- .../org/apache/comet/serde/QueryPlanSerde.scala | 12 +++++------- .../test/scala/org/apache/comet/CometCastSuite.scala | 6 ++++-- 3 files changed, 16 insertions(+), 13 deletions(-) diff --git a/docs/source/user-guide/compatibility.md b/docs/source/user-guide/compatibility.md index 3cb6bb4d8..ec1fe58af 100644 --- a/docs/source/user-guide/compatibility.md +++ b/docs/source/user-guide/compatibility.md @@ -36,14 +36,17 @@ There is an [epic](https://github.com/apache/datafusion-comet/issues/313) where Cast operations in Comet fall into three levels of support: -- Compatible: The results match Apache Spark -- Incompatible: The results may match with Apache Spark for some inputs, but there are known issues where some - inputs will result in incorrect results or exceptions. The query stage will fall back to Spark by default. Setting +- **Compatible**: The results match Apache Spark +- **Incompatible**: The results may match Apache Spark for some inputs, but there are known issues where some inputs + will result in incorrect results or exceptions. The query stage will fall back to Spark by default. Setting `spark.comet.cast.allowIncompatible=true` will allow all incompatible casts to run natively in Comet, but this is not recommended for production use. -- Unsupported: Comet does not provide a native version of this cast expression and the query stage will fall back to +- **Unsupported**: Comet does not provide a native version of this cast expression and the query stage will fall back to Spark. +The following table shows the current cast operations supported by Comet. Any cast that does not appear in this +table (such as those involving complex types and timestamp_ntz, for example) are not supported by Comet. + | From Type | To Type | Compatible? | Notes | | --------- | --------- | ------------ | ----------------------------------- | | boolean | boolean | Compatible | | 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 f0406234b..a06943636 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -591,9 +591,9 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { def getIncompatMessage(reason: Option[String]) = s"Comet does not guarantee correct results for cast " + - s"from ${child.dataType} to $dt " + - s"with timezone $timeZoneId and evalMode $evalModeStr" + - reason.map(str => s" ($str)").getOrElse("") + s"from ${child.dataType} to $dt " + + s"with timezone $timeZoneId and evalMode $evalModeStr" + + reason.map(str => s" ($str)").getOrElse("") castSupport match { case Compatible => @@ -606,16 +606,14 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { withInfo( expr, s"${getIncompatMessage(reason)}. To enable all incompatible casts, set " + - s"${CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key}=true" - ) + s"${CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key}=true") None } case Unsupported => withInfo( expr, s"Unsupported cast from ${child.dataType} to $dt " + - s"with timezone $timeZoneId and evalMode $evalModeStr" - ) + s"with timezone $timeZoneId and evalMode $evalModeStr") None } } else { diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index f9c426125..dbcc009b9 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -19,10 +19,10 @@ package org.apache.comet -import org.apache.comet.expressions.{CometCast, Compatible, Incompatible, Unsupported} - import java.io.File + import scala.util.Random + import org.apache.spark.sql.{CometTestBase, DataFrame, SaveMode} import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -30,6 +30,8 @@ import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, DataTypes} +import org.apache.comet.expressions.{CometCast, Compatible, Incompatible, Unsupported} + class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { import testImplicits._ From ee77b1472317150e927f5acb9af7d6ab13ecb9c1 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 14:05:18 -0600 Subject: [PATCH 21/38] fix lint error --- docs/source/user-guide/configs.md | 2 +- spark/src/test/scala/org/apache/comet/CometCastSuite.scala | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 138e26cf2..570723626 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -25,7 +25,7 @@ Comet provides the following configuration settings. |--------|-------------|---------------| | spark.comet.ansi.enabled | Comet does not respect ANSI mode in most cases and by default will not accelerate queries when ansi mode is enabled. Enable this setting to test Comet's experimental support for ANSI mode. This should not be used in production. | false | | spark.comet.batchSize | The columnar batch size, i.e., the maximum number of rows that a batch can contain. | 8192 | -| spark.comet.cast.allowIncompatible | Comet is not currently fully compatible with Spark for all cast operations. | true | +| spark.comet.cast.allowIncompatible | Comet is not currently fully compatible with Spark for all cast operations. Set this config to true to allow them anyway. See compatibility guide for more information. | true | | 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 | diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index dbcc009b9..1af7f9af7 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -92,10 +92,11 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { assertTestsExist(supportedTypes, supportedTypes) } + // TODO automate writing this documentation in maven build test("generate cast documentation") { // scalastyle:on println - println(s"| From Type | To Type | Compatible? | Notes |") - println(s"|-|-|-|-|") + println("| From Type | To Type | Compatible? | Notes |") + println("|-|-|-|-|") for (fromType <- supportedTypes) { for (toType <- supportedTypes) { if (Cast.canCast(fromType, toType)) { From 6d903c39d50ccc5eb9698adf6ddbf204fd5a62a3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 14:18:38 -0600 Subject: [PATCH 22/38] automate docs --- .../scala/org/apache/comet/CometConf.scala | 33 ---- .../user-guide/compatibility-template.md | 50 +++++ docs/source/user-guide/compatibility.md | 171 ------------------ spark/pom.xml | 4 - .../scala/org/apache/comet/GenerateDocs.scala | 73 ++++++++ .../apache/comet/expressions/CometCast.scala | 16 ++ .../org/apache/comet/CometCastSuite.scala | 45 +---- 7 files changed, 140 insertions(+), 252 deletions(-) create mode 100644 docs/source/user-guide/compatibility-template.md delete mode 100644 docs/source/user-guide/compatibility.md create mode 100644 spark/src/main/scala/org/apache/comet/GenerateDocs.scala diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 14360ee66..dc485da52 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -628,36 +628,3 @@ private[comet] case class ConfigBuilder(key: String) { private object ConfigEntry { val UNDEFINED = "" } - -/** - * Utility for generating markdown documentation from the configs. - * - * This is invoked when running `mvn clean package -DskipTests`. - */ -object CometConfGenerateDocs { - def main(args: Array[String]): Unit = { - if (args.length != 2) { - // scalastyle:off println - println("Missing arguments for template file and output file") - // scalastyle:on println - sys.exit(-1) - } - val templateFilename = args.head - val outputFilename = args(1) - val w = new BufferedOutputStream(new FileOutputStream(outputFilename)) - for (line <- Source.fromFile(templateFilename).getLines()) { - if (line.trim == "") { - val publicConfigs = CometConf.allConfs.filter(_.isPublic) - val confs = publicConfigs.sortBy(_.key) - w.write("| Config | Description | Default Value |\n".getBytes) - w.write("|--------|-------------|---------------|\n".getBytes) - for (conf <- confs) { - w.write(s"| ${conf.key} | ${conf.doc.trim} | ${conf.defaultValueString} |\n".getBytes) - } - } else { - w.write(s"${line.trim}\n".getBytes) - } - } - w.close() - } -} diff --git a/docs/source/user-guide/compatibility-template.md b/docs/source/user-guide/compatibility-template.md new file mode 100644 index 000000000..ea00ba140 --- /dev/null +++ b/docs/source/user-guide/compatibility-template.md @@ -0,0 +1,50 @@ + + +# Compatibility Guide + +Comet aims to provide consistent results with the version of Apache Spark that is being used. + +This guide offers information about areas of functionality where there are known differences. + +## ANSI mode + +Comet currently ignores ANSI mode in most cases, and therefore can produce different results than Spark. By default, +Comet will fall back to Spark if ANSI mode is enabled. To enable Comet to accelerate queries when ANSI mode is enabled, +specify `spark.comet.ansi.enabled=true` in the Spark configuration. Comet's ANSI support is experimental and should not +be used in production. + +There is an [epic](https://github.com/apache/datafusion-comet/issues/313) where we are tracking the work to fully implement ANSI support. + +## Cast + +Cast operations in Comet fall into three levels of support: + +- **Compatible**: The results match Apache Spark +- **Incompatible**: The results may match Apache Spark for some inputs, but there are known issues where some inputs + will result in incorrect results or exceptions. The query stage will fall back to Spark by default. Setting + `spark.comet.cast.allowIncompatible=true` will allow all incompatible casts to run natively in Comet, but this is not + recommended for production use. +- **Unsupported**: Comet does not provide a native version of this cast expression and the query stage will fall back to + Spark. + +The following table shows the current cast operations supported by Comet. Any cast that does not appear in this +table (such as those involving complex types and timestamp_ntz, for example) are not supported by Comet. + + \ No newline at end of file diff --git a/docs/source/user-guide/compatibility.md b/docs/source/user-guide/compatibility.md deleted file mode 100644 index ec1fe58af..000000000 --- a/docs/source/user-guide/compatibility.md +++ /dev/null @@ -1,171 +0,0 @@ - - -# Compatibility Guide - -Comet aims to provide consistent results with the version of Apache Spark that is being used. - -This guide offers information about areas of functionality where there are known differences. - -## ANSI mode - -Comet currently ignores ANSI mode in most cases, and therefore can produce different results than Spark. By default, -Comet will fall back to Spark if ANSI mode is enabled. To enable Comet to accelerate queries when ANSI mode is enabled, -specify `spark.comet.ansi.enabled=true` in the Spark configuration. Comet's ANSI support is experimental and should not -be used in production. - -There is an [epic](https://github.com/apache/datafusion-comet/issues/313) where we are tracking the work to fully implement ANSI support. - -## Cast - -Cast operations in Comet fall into three levels of support: - -- **Compatible**: The results match Apache Spark -- **Incompatible**: The results may match Apache Spark for some inputs, but there are known issues where some inputs - will result in incorrect results or exceptions. The query stage will fall back to Spark by default. Setting - `spark.comet.cast.allowIncompatible=true` will allow all incompatible casts to run natively in Comet, but this is not - recommended for production use. -- **Unsupported**: Comet does not provide a native version of this cast expression and the query stage will fall back to - Spark. - -The following table shows the current cast operations supported by Comet. Any cast that does not appear in this -table (such as those involving complex types and timestamp_ntz, for example) are not supported by Comet. - -| From Type | To Type | Compatible? | Notes | -| --------- | --------- | ------------ | ----------------------------------- | -| boolean | boolean | Compatible | | -| boolean | byte | Compatible | | -| boolean | short | Compatible | | -| boolean | integer | Compatible | | -| boolean | long | Compatible | | -| boolean | float | Compatible | | -| boolean | double | Compatible | | -| boolean | decimal | Unsupported | | -| boolean | string | Compatible | | -| boolean | timestamp | Unsupported | | -| byte | boolean | Compatible | | -| byte | byte | Compatible | | -| byte | short | Compatible | | -| byte | integer | Compatible | | -| byte | long | Compatible | | -| byte | float | Compatible | | -| byte | double | Compatible | | -| byte | decimal | Compatible | | -| byte | string | Compatible | | -| byte | binary | Unsupported | | -| byte | timestamp | Unsupported | | -| short | boolean | Compatible | | -| short | byte | Compatible | | -| short | short | Compatible | | -| short | integer | Compatible | | -| short | long | Compatible | | -| short | float | Compatible | | -| short | double | Compatible | | -| short | decimal | Compatible | | -| short | string | Compatible | | -| short | binary | Unsupported | | -| short | timestamp | Unsupported | | -| integer | boolean | Compatible | | -| integer | byte | Compatible | | -| integer | short | Compatible | | -| integer | integer | Compatible | | -| integer | long | Compatible | | -| integer | float | Compatible | | -| integer | double | Compatible | | -| integer | decimal | Compatible | | -| integer | string | Compatible | | -| integer | binary | Unsupported | | -| integer | timestamp | Unsupported | | -| long | boolean | Compatible | | -| long | byte | Compatible | | -| long | short | Compatible | | -| long | integer | Compatible | | -| long | long | Compatible | | -| long | float | Compatible | | -| long | double | Compatible | | -| long | decimal | Compatible | | -| long | string | Compatible | | -| long | binary | Unsupported | | -| long | timestamp | Unsupported | | -| float | boolean | Compatible | | -| float | byte | Unsupported | | -| float | short | Unsupported | | -| float | integer | Unsupported | | -| float | long | Unsupported | | -| float | float | Compatible | | -| float | double | Compatible | | -| float | decimal | Unsupported | | -| float | string | Incompatible | | -| float | timestamp | Unsupported | | -| double | boolean | Compatible | | -| double | byte | Unsupported | | -| double | short | Unsupported | | -| double | integer | Unsupported | | -| double | long | Unsupported | | -| double | float | Compatible | | -| double | double | Compatible | | -| double | decimal | Incompatible | | -| double | string | Incompatible | | -| double | timestamp | Unsupported | | -| decimal | boolean | Unsupported | | -| decimal | byte | Unsupported | | -| decimal | short | Unsupported | | -| decimal | integer | Unsupported | | -| decimal | long | Unsupported | | -| decimal | float | Compatible | | -| decimal | double | Compatible | | -| decimal | decimal | Compatible | | -| decimal | string | Unsupported | | -| decimal | timestamp | Unsupported | | -| string | boolean | Compatible | | -| string | byte | Compatible | | -| string | short | Compatible | | -| string | integer | Compatible | | -| string | long | Compatible | | -| string | float | Unsupported | | -| string | double | Unsupported | | -| string | decimal | Unsupported | | -| string | string | Compatible | | -| string | binary | Compatible | | -| string | date | Unsupported | | -| string | timestamp | Incompatible | Not all valid formats are supported | -| binary | string | Incompatible | | -| binary | binary | Compatible | | -| date | boolean | Unsupported | | -| date | byte | Unsupported | | -| date | short | Unsupported | | -| date | integer | Unsupported | | -| date | long | Unsupported | | -| date | float | Unsupported | | -| date | double | Unsupported | | -| date | decimal | Unsupported | | -| date | string | Compatible | | -| date | date | Compatible | | -| date | timestamp | Unsupported | | -| timestamp | boolean | Unsupported | | -| timestamp | byte | Unsupported | | -| timestamp | short | Unsupported | | -| timestamp | integer | Unsupported | | -| timestamp | long | Compatible | | -| timestamp | float | Unsupported | | -| timestamp | double | Unsupported | | -| timestamp | decimal | Unsupported | | -| timestamp | string | Compatible | | -| timestamp | date | Compatible | | -| timestamp | timestamp | Compatible | | diff --git a/spark/pom.xml b/spark/pom.xml index 9392b7fe9..ec88c9a36 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -277,10 +277,6 @@ under the License. org.apache.comet.CometConfGenerateDocs - - docs/source/user-guide/configs-template.md - docs/source/user-guide/configs.md - compile diff --git a/spark/src/main/scala/org/apache/comet/GenerateDocs.scala b/spark/src/main/scala/org/apache/comet/GenerateDocs.scala new file mode 100644 index 000000000..87f298435 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/GenerateDocs.scala @@ -0,0 +1,73 @@ +package org.apache.comet + +import org.apache.comet.expressions.{CometCast, Compatible, Incompatible, Unsupported} +import org.apache.spark.sql.catalyst.expressions.Cast + +import java.io.{BufferedOutputStream, FileOutputStream} +import scala.io.Source + +/** + * Utility for generating markdown documentation from the configs. + * + * This is invoked when running `mvn clean package -DskipTests`. + */ +object GenerateDocs { + + def main(args: Array[String]): Unit = { + generateConfigReference() + generateCompatReference() + } + + def generateConfigReference(): Unit = { + val templateFilename = "docs/source/user-guide/configs-template.md" + val outputFilename = "docs/source/user-guide/configs.md" + val w = new BufferedOutputStream(new FileOutputStream(outputFilename)) + for (line <- Source.fromFile(templateFilename).getLines()) { + if (line.trim == "") { + val publicConfigs = CometConf.allConfs.filter(_.isPublic) + val confs = publicConfigs.sortBy(_.key) + w.write("| Config | Description | Default Value |\n".getBytes) + w.write("|--------|-------------|---------------|\n".getBytes) + for (conf <- confs) { + w.write(s"| ${conf.key} | ${conf.doc.trim} | ${conf.defaultValueString} |\n".getBytes) + } + } else { + w.write(s"${line.trim}\n".getBytes) + } + } + w.close() + } + + def generateCompatReference(): Unit = { + val templateFilename = "docs/source/user-guide/compatibility-template.md" + val outputFilename = "docs/source/user-guide/compatibility.md" + val w = new BufferedOutputStream(new FileOutputStream(outputFilename)) + for (line <- Source.fromFile(templateFilename).getLines()) { + if (line.trim == "") { + w.write("| From Type | To Type | Compatible? | Notes |\n".getBytes) + w.write("|-|-|-|-|\n".getBytes) + for (fromType <- CometCast.supportedTypes) { + for (toType <- CometCast.supportedTypes) { + if (Cast.canCast(fromType, toType) && fromType != toType) { + val fromTypeName = fromType.typeName.replace("(10,2)", "") + val toTypeName = toType.typeName.replace("(10,2)", "") + CometCast.isSupported(fromType, toType, None, "LEGACY") match { + case Compatible => + w.write(s"| $fromTypeName | $toTypeName | Compatible | |".getBytes) + case Incompatible(Some(reason)) => + w.write(s"| $fromTypeName | $toTypeName | Incompatible | $reason |".getBytes) + case Incompatible(None) => + w.write(s"| $fromTypeName | $toTypeName | Incompatible | |.getBytes") + case Unsupported => + w.write(s"| $fromTypeName | $toTypeName | Unsupported | |".getBytes) + } + } + } + } + } else { + w.write(s"${line.trim}\n".getBytes) + } + } + w.close() + } +} 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 16782a4bd..6c77b1983 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -36,6 +36,22 @@ object Unsupported extends SupportLevel object CometCast { + val supportedTypes = + Seq( + DataTypes.BooleanType, + DataTypes.ByteType, + DataTypes.ShortType, + DataTypes.IntegerType, + DataTypes.LongType, + DataTypes.FloatType, + DataTypes.DoubleType, + DataTypes.createDecimalType(10, 2), + DataTypes.StringType, + DataTypes.BinaryType, + DataTypes.DateType, + DataTypes.TimestampType) + // TODO add DataTypes.TimestampNTZType for Spark 3.4 and later + def isSupported( fromType: DataType, toType: DataType, diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 1af7f9af7..d9ec99d55 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -51,23 +51,6 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { private val datePattern = "0123456789/" + whitespaceChars private val timestampPattern = "0123456789/:T" + whitespaceChars - // make sure we have tests for all combinations of our supported types - val supportedTypes = - Seq( - DataTypes.BooleanType, - DataTypes.ByteType, - DataTypes.ShortType, - DataTypes.IntegerType, - DataTypes.LongType, - DataTypes.FloatType, - DataTypes.DoubleType, - DataTypes.createDecimalType(10, 2), - DataTypes.StringType, - DataTypes.BinaryType, - DataTypes.DateType, - DataTypes.TimestampType) - // TODO add DataTypes.TimestampNTZType for Spark 3.4 and later - test("all valid cast combinations covered") { val names = testNames @@ -89,33 +72,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - assertTestsExist(supportedTypes, supportedTypes) - } - - // TODO automate writing this documentation in maven build - test("generate cast documentation") { - // scalastyle:on println - println("| From Type | To Type | Compatible? | Notes |") - println("|-|-|-|-|") - for (fromType <- supportedTypes) { - for (toType <- supportedTypes) { - if (Cast.canCast(fromType, toType)) { - val fromTypeName = fromType.typeName.replace("(10,2)", "") - val toTypeName = toType.typeName.replace("(10,2)", "") - CometCast.isSupported(fromType, toType, None, "LEGACY") match { - case Compatible => - println(s"| $fromTypeName | $toTypeName | Compatible | |") - case Incompatible(Some(reason)) => - println(s"| $fromTypeName | $toTypeName | Incompatible | $reason |") - case Incompatible(None) => - println(s"| $fromTypeName | $toTypeName | Incompatible | |") - case Unsupported => - println(s"| $fromTypeName | $toTypeName | Unsupported | |") - } - } - } - } - // scalastyle:off println + assertTestsExist(CometCast.supportedTypes, CometCast.supportedTypes) } // CAST from BooleanType From 4c24e41fcc47ddb9e4a8495c442b4fc1e2abd618 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 14:38:53 -0600 Subject: [PATCH 23/38] automate docs --- docs/source/user-guide/compatibility.md | 159 ++++++++++++++++++ spark/pom.xml | 9 +- .../scala/org/apache/comet/GenerateDocs.scala | 41 +++-- .../apache/comet/expressions/CometCast.scala | 2 +- .../apache/comet/serde/QueryPlanSerde.scala | 4 +- 5 files changed, 200 insertions(+), 15 deletions(-) create mode 100644 docs/source/user-guide/compatibility.md diff --git a/docs/source/user-guide/compatibility.md b/docs/source/user-guide/compatibility.md new file mode 100644 index 000000000..9c57d3d19 --- /dev/null +++ b/docs/source/user-guide/compatibility.md @@ -0,0 +1,159 @@ + + +# Compatibility Guide + +Comet aims to provide consistent results with the version of Apache Spark that is being used. + +This guide offers information about areas of functionality where there are known differences. + +## ANSI mode + +Comet currently ignores ANSI mode in most cases, and therefore can produce different results than Spark. By default, +Comet will fall back to Spark if ANSI mode is enabled. To enable Comet to accelerate queries when ANSI mode is enabled, +specify `spark.comet.ansi.enabled=true` in the Spark configuration. Comet's ANSI support is experimental and should not +be used in production. + +There is an [epic](https://github.com/apache/datafusion-comet/issues/313) where we are tracking the work to fully implement ANSI support. + +## Cast + +Cast operations in Comet fall into three levels of support: + +- **Compatible**: The results match Apache Spark +- **Incompatible**: The results may match Apache Spark for some inputs, but there are known issues where some inputs +will result in incorrect results or exceptions. The query stage will fall back to Spark by default. Setting +`spark.comet.cast.allowIncompatible=true` will allow all incompatible casts to run natively in Comet, but this is not +recommended for production use. +- **Unsupported**: Comet does not provide a native version of this cast expression and the query stage will fall back to +Spark. + +The following table shows the current cast operations supported by Comet. Any cast that does not appear in this +table (such as those involving complex types and timestamp_ntz, for example) are not supported by Comet. + +| From Type | To Type | Compatible? | Notes | +|-|-|-|-| +| boolean | byte | Compatible | | +| boolean | short | Compatible | | +| boolean | integer | Compatible | | +| boolean | long | Compatible | | +| boolean | float | Compatible | | +| boolean | double | Compatible | | +| boolean | decimal | Unsupported | | +| boolean | string | Compatible | | +| boolean | timestamp | Unsupported | | +| byte | boolean | Compatible | | +| byte | short | Compatible | | +| byte | integer | Compatible | | +| byte | long | Compatible | | +| byte | float | Compatible | | +| byte | double | Compatible | | +| byte | decimal | Compatible | | +| byte | string | Compatible | | +| byte | binary | Unsupported | | +| byte | timestamp | Unsupported | | +| short | boolean | Compatible | | +| short | byte | Compatible | | +| short | integer | Compatible | | +| short | long | Compatible | | +| short | float | Compatible | | +| short | double | Compatible | | +| short | decimal | Compatible | | +| short | string | Compatible | | +| short | binary | Unsupported | | +| short | timestamp | Unsupported | | +| integer | boolean | Compatible | | +| integer | byte | Compatible | | +| integer | short | Compatible | | +| integer | long | Compatible | | +| integer | float | Compatible | | +| integer | double | Compatible | | +| integer | decimal | Compatible | | +| integer | string | Compatible | | +| integer | binary | Unsupported | | +| integer | timestamp | Unsupported | | +| long | boolean | Compatible | | +| long | byte | Compatible | | +| long | short | Compatible | | +| long | integer | Compatible | | +| long | float | Compatible | | +| long | double | Compatible | | +| long | decimal | Compatible | | +| long | string | Compatible | | +| long | binary | Unsupported | | +| long | timestamp | Unsupported | | +| float | boolean | Compatible | | +| float | byte | Unsupported | | +| float | short | Unsupported | | +| float | integer | Unsupported | | +| float | long | Unsupported | | +| float | double | Compatible | | +| float | decimal | Unsupported | | +| float | string | Incompatible | | +| float | timestamp | Unsupported | | +| double | boolean | Compatible | | +| double | byte | Unsupported | | +| double | short | Unsupported | | +| double | integer | Unsupported | | +| double | long | Unsupported | | +| double | float | Compatible | | +| double | decimal | Incompatible | | +| double | string | Incompatible | | +| double | timestamp | Unsupported | | +| decimal | boolean | Unsupported | | +| decimal | byte | Unsupported | | +| decimal | short | Unsupported | | +| decimal | integer | Unsupported | | +| decimal | long | Unsupported | | +| decimal | float | Compatible | | +| decimal | double | Compatible | | +| decimal | string | Unsupported | | +| decimal | timestamp | Unsupported | | +| string | boolean | Compatible | | +| string | byte | Compatible | | +| string | short | Compatible | | +| string | integer | Compatible | | +| string | long | Compatible | | +| string | float | Unsupported | | +| string | double | Unsupported | | +| string | decimal | Unsupported | | +| string | binary | Compatible | | +| string | date | Unsupported | | +| string | timestamp | Incompatible | Not all valid formats are supported | +| binary | string | Incompatible | | +| date | boolean | Unsupported | | +| date | byte | Unsupported | | +| date | short | Unsupported | | +| date | integer | Unsupported | | +| date | long | Unsupported | | +| date | float | Unsupported | | +| date | double | Unsupported | | +| date | decimal | Unsupported | | +| date | string | Compatible | | +| date | timestamp | Unsupported | | +| timestamp | boolean | Unsupported | | +| timestamp | byte | Unsupported | | +| timestamp | short | Unsupported | | +| timestamp | integer | Unsupported | | +| timestamp | long | Compatible | | +| timestamp | float | Unsupported | | +| timestamp | double | Unsupported | | +| timestamp | decimal | Unsupported | | +| timestamp | string | Compatible | | +| timestamp | date | Compatible | | diff --git a/spark/pom.xml b/spark/pom.xml index ec88c9a36..7d3d3d758 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -58,6 +58,11 @@ under the License. org.scala-lang scala-library + + org.scala-lang + scala-reflect + provided + com.google.protobuf protobuf-java @@ -270,13 +275,13 @@ under the License. 3.2.0 - generate-config-docs + generate-user-guide-reference-docs package java - org.apache.comet.CometConfGenerateDocs + org.apache.comet.GenerateDocs compile diff --git a/spark/src/main/scala/org/apache/comet/GenerateDocs.scala b/spark/src/main/scala/org/apache/comet/GenerateDocs.scala index 87f298435..8c414c7fe 100644 --- a/spark/src/main/scala/org/apache/comet/GenerateDocs.scala +++ b/spark/src/main/scala/org/apache/comet/GenerateDocs.scala @@ -1,11 +1,32 @@ -package org.apache.comet +/* + * 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. + */ -import org.apache.comet.expressions.{CometCast, Compatible, Incompatible, Unsupported} -import org.apache.spark.sql.catalyst.expressions.Cast +package org.apache.comet import java.io.{BufferedOutputStream, FileOutputStream} + import scala.io.Source +import org.apache.spark.sql.catalyst.expressions.Cast + +import org.apache.comet.expressions.{CometCast, Compatible, Incompatible, Unsupported} + /** * Utility for generating markdown documentation from the configs. * @@ -15,10 +36,10 @@ object GenerateDocs { def main(args: Array[String]): Unit = { generateConfigReference() - generateCompatReference() + generateCompatibilityGuide() } - def generateConfigReference(): Unit = { + private def generateConfigReference(): Unit = { val templateFilename = "docs/source/user-guide/configs-template.md" val outputFilename = "docs/source/user-guide/configs.md" val w = new BufferedOutputStream(new FileOutputStream(outputFilename)) @@ -38,7 +59,7 @@ object GenerateDocs { w.close() } - def generateCompatReference(): Unit = { + private def generateCompatibilityGuide(): Unit = { val templateFilename = "docs/source/user-guide/compatibility-template.md" val outputFilename = "docs/source/user-guide/compatibility.md" val w = new BufferedOutputStream(new FileOutputStream(outputFilename)) @@ -53,13 +74,13 @@ object GenerateDocs { val toTypeName = toType.typeName.replace("(10,2)", "") CometCast.isSupported(fromType, toType, None, "LEGACY") match { case Compatible => - w.write(s"| $fromTypeName | $toTypeName | Compatible | |".getBytes) + w.write(s"| $fromTypeName | $toTypeName | Compatible | |\n".getBytes) case Incompatible(Some(reason)) => - w.write(s"| $fromTypeName | $toTypeName | Incompatible | $reason |".getBytes) + w.write(s"| $fromTypeName | $toTypeName | Incompatible | $reason |\n".getBytes) case Incompatible(None) => - w.write(s"| $fromTypeName | $toTypeName | Incompatible | |.getBytes") + w.write(s"| $fromTypeName | $toTypeName | Incompatible | |\n".getBytes) case Unsupported => - w.write(s"| $fromTypeName | $toTypeName | Unsupported | |".getBytes) + w.write(s"| $fromTypeName | $toTypeName | Unsupported | |\n".getBytes) } } } 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 6c77b1983..d3adcd077 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -36,7 +36,7 @@ object Unsupported extends SupportLevel object CometCast { - val supportedTypes = + def supportedTypes: Seq[DataType] = Seq( DataTypes.BooleanType, DataTypes.ByteType, 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 a06943636..e77adc9bb 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -576,7 +576,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { val value = cast.eval() exprToProtoInternal(Literal(value, dataType), inputs) - case cast @ Cast(child, dt, timeZoneId, evalMode) => + case Cast(child, dt, timeZoneId, evalMode) => val childExpr = exprToProtoInternal(child, inputs) if (childExpr.isDefined) { val evalModeStr = if (evalMode.isInstanceOf[Boolean]) { @@ -590,7 +590,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde { CometCast.isSupported(child.dataType, dt, timeZoneId, evalModeStr) def getIncompatMessage(reason: Option[String]) = - s"Comet does not guarantee correct results for cast " + + "Comet does not guarantee correct results for cast " + s"from ${child.dataType} to $dt " + s"with timezone $timeZoneId and evalMode $evalModeStr" + reason.map(str => s" ($str)").getOrElse("") From b8df40fcdceb18328721c04afc797d196c72bb42 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 14:47:49 -0600 Subject: [PATCH 24/38] remove unused imports --- common/src/main/scala/org/apache/comet/CometConf.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index dc485da52..ec809bde4 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -19,11 +19,9 @@ package org.apache.comet -import java.io.{BufferedOutputStream, FileOutputStream} import java.util.concurrent.TimeUnit import scala.collection.mutable.ListBuffer -import scala.io.Source import org.apache.spark.network.util.ByteUnit import org.apache.spark.network.util.JavaUtils From 4b98bc23e1ceb961e752e6428041cb1ad96aab34 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 14:54:43 -0600 Subject: [PATCH 25/38] make format --- .../main/scala/org/apache/comet/expressions/CometCast.scala | 4 +--- spark/src/test/scala/org/apache/comet/CometCastSuite.scala | 2 +- 2 files changed, 2 insertions(+), 4 deletions(-) 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 d3adcd077..d37bcd8e7 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -19,8 +19,6 @@ package org.apache.comet.expressions -import org.apache.spark.sql.catalyst.expressions.Cast -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, DataTypes, DecimalType} sealed trait SupportLevel @@ -122,7 +120,7 @@ object CometCast { 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" => - Incompatible(Some(s"ANSI mode not supported")) + Incompatible(Some("ANSI mode not supported")) case DataTypes.TimestampType => // https://github.com/apache/datafusion-comet/issues/328 Incompatible(Some("Not all valid formats are supported")) diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index d9ec99d55..267e03659 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, DataTypes} -import org.apache.comet.expressions.{CometCast, Compatible, Incompatible, Unsupported} +import org.apache.comet.expressions.CometCast class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { import testImplicits._ From 6675c72130cba80665660096483f630ae33956f0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 15:00:03 -0600 Subject: [PATCH 26/38] prettier --- .../user-guide/compatibility-template.md | 2 +- docs/source/user-guide/compatibility.md | 228 +++++++++--------- 2 files changed, 115 insertions(+), 115 deletions(-) diff --git a/docs/source/user-guide/compatibility-template.md b/docs/source/user-guide/compatibility-template.md index ea00ba140..deaca2d24 100644 --- a/docs/source/user-guide/compatibility-template.md +++ b/docs/source/user-guide/compatibility-template.md @@ -47,4 +47,4 @@ Cast operations in Comet fall into three levels of support: The following table shows the current cast operations supported by Comet. Any cast that does not appear in this table (such as those involving complex types and timestamp_ntz, for example) are not supported by Comet. - \ No newline at end of file + diff --git a/docs/source/user-guide/compatibility.md b/docs/source/user-guide/compatibility.md index 9c57d3d19..9a2478d37 100644 --- a/docs/source/user-guide/compatibility.md +++ b/docs/source/user-guide/compatibility.md @@ -38,122 +38,122 @@ Cast operations in Comet fall into three levels of support: - **Compatible**: The results match Apache Spark - **Incompatible**: The results may match Apache Spark for some inputs, but there are known issues where some inputs -will result in incorrect results or exceptions. The query stage will fall back to Spark by default. Setting -`spark.comet.cast.allowIncompatible=true` will allow all incompatible casts to run natively in Comet, but this is not -recommended for production use. + will result in incorrect results or exceptions. The query stage will fall back to Spark by default. Setting + `spark.comet.cast.allowIncompatible=true` will allow all incompatible casts to run natively in Comet, but this is not + recommended for production use. - **Unsupported**: Comet does not provide a native version of this cast expression and the query stage will fall back to -Spark. + Spark. The following table shows the current cast operations supported by Comet. Any cast that does not appear in this table (such as those involving complex types and timestamp_ntz, for example) are not supported by Comet. -| From Type | To Type | Compatible? | Notes | -|-|-|-|-| -| boolean | byte | Compatible | | -| boolean | short | Compatible | | -| boolean | integer | Compatible | | -| boolean | long | Compatible | | -| boolean | float | Compatible | | -| boolean | double | Compatible | | -| boolean | decimal | Unsupported | | -| boolean | string | Compatible | | -| boolean | timestamp | Unsupported | | -| byte | boolean | Compatible | | -| byte | short | Compatible | | -| byte | integer | Compatible | | -| byte | long | Compatible | | -| byte | float | Compatible | | -| byte | double | Compatible | | -| byte | decimal | Compatible | | -| byte | string | Compatible | | -| byte | binary | Unsupported | | -| byte | timestamp | Unsupported | | -| short | boolean | Compatible | | -| short | byte | Compatible | | -| short | integer | Compatible | | -| short | long | Compatible | | -| short | float | Compatible | | -| short | double | Compatible | | -| short | decimal | Compatible | | -| short | string | Compatible | | -| short | binary | Unsupported | | -| short | timestamp | Unsupported | | -| integer | boolean | Compatible | | -| integer | byte | Compatible | | -| integer | short | Compatible | | -| integer | long | Compatible | | -| integer | float | Compatible | | -| integer | double | Compatible | | -| integer | decimal | Compatible | | -| integer | string | Compatible | | -| integer | binary | Unsupported | | -| integer | timestamp | Unsupported | | -| long | boolean | Compatible | | -| long | byte | Compatible | | -| long | short | Compatible | | -| long | integer | Compatible | | -| long | float | Compatible | | -| long | double | Compatible | | -| long | decimal | Compatible | | -| long | string | Compatible | | -| long | binary | Unsupported | | -| long | timestamp | Unsupported | | -| float | boolean | Compatible | | -| float | byte | Unsupported | | -| float | short | Unsupported | | -| float | integer | Unsupported | | -| float | long | Unsupported | | -| float | double | Compatible | | -| float | decimal | Unsupported | | -| float | string | Incompatible | | -| float | timestamp | Unsupported | | -| double | boolean | Compatible | | -| double | byte | Unsupported | | -| double | short | Unsupported | | -| double | integer | Unsupported | | -| double | long | Unsupported | | -| double | float | Compatible | | -| double | decimal | Incompatible | | -| double | string | Incompatible | | -| double | timestamp | Unsupported | | -| decimal | boolean | Unsupported | | -| decimal | byte | Unsupported | | -| decimal | short | Unsupported | | -| decimal | integer | Unsupported | | -| decimal | long | Unsupported | | -| decimal | float | Compatible | | -| decimal | double | Compatible | | -| decimal | string | Unsupported | | -| decimal | timestamp | Unsupported | | -| string | boolean | Compatible | | -| string | byte | Compatible | | -| string | short | Compatible | | -| string | integer | Compatible | | -| string | long | Compatible | | -| string | float | Unsupported | | -| string | double | Unsupported | | -| string | decimal | Unsupported | | -| string | binary | Compatible | | -| string | date | Unsupported | | -| string | timestamp | Incompatible | Not all valid formats are supported | -| binary | string | Incompatible | | -| date | boolean | Unsupported | | -| date | byte | Unsupported | | -| date | short | Unsupported | | -| date | integer | Unsupported | | -| date | long | Unsupported | | -| date | float | Unsupported | | -| date | double | Unsupported | | -| date | decimal | Unsupported | | -| date | string | Compatible | | -| date | timestamp | Unsupported | | -| timestamp | boolean | Unsupported | | -| timestamp | byte | Unsupported | | -| timestamp | short | Unsupported | | -| timestamp | integer | Unsupported | | -| timestamp | long | Compatible | | -| timestamp | float | Unsupported | | -| timestamp | double | Unsupported | | -| timestamp | decimal | Unsupported | | -| timestamp | string | Compatible | | -| timestamp | date | Compatible | | +| From Type | To Type | Compatible? | Notes | +| --------- | --------- | ------------ | ----------------------------------- | +| boolean | byte | Compatible | | +| boolean | short | Compatible | | +| boolean | integer | Compatible | | +| boolean | long | Compatible | | +| boolean | float | Compatible | | +| boolean | double | Compatible | | +| boolean | decimal | Unsupported | | +| boolean | string | Compatible | | +| boolean | timestamp | Unsupported | | +| byte | boolean | Compatible | | +| byte | short | Compatible | | +| byte | integer | Compatible | | +| byte | long | Compatible | | +| byte | float | Compatible | | +| byte | double | Compatible | | +| byte | decimal | Compatible | | +| byte | string | Compatible | | +| byte | binary | Unsupported | | +| byte | timestamp | Unsupported | | +| short | boolean | Compatible | | +| short | byte | Compatible | | +| short | integer | Compatible | | +| short | long | Compatible | | +| short | float | Compatible | | +| short | double | Compatible | | +| short | decimal | Compatible | | +| short | string | Compatible | | +| short | binary | Unsupported | | +| short | timestamp | Unsupported | | +| integer | boolean | Compatible | | +| integer | byte | Compatible | | +| integer | short | Compatible | | +| integer | long | Compatible | | +| integer | float | Compatible | | +| integer | double | Compatible | | +| integer | decimal | Compatible | | +| integer | string | Compatible | | +| integer | binary | Unsupported | | +| integer | timestamp | Unsupported | | +| long | boolean | Compatible | | +| long | byte | Compatible | | +| long | short | Compatible | | +| long | integer | Compatible | | +| long | float | Compatible | | +| long | double | Compatible | | +| long | decimal | Compatible | | +| long | string | Compatible | | +| long | binary | Unsupported | | +| long | timestamp | Unsupported | | +| float | boolean | Compatible | | +| float | byte | Unsupported | | +| float | short | Unsupported | | +| float | integer | Unsupported | | +| float | long | Unsupported | | +| float | double | Compatible | | +| float | decimal | Unsupported | | +| float | string | Incompatible | | +| float | timestamp | Unsupported | | +| double | boolean | Compatible | | +| double | byte | Unsupported | | +| double | short | Unsupported | | +| double | integer | Unsupported | | +| double | long | Unsupported | | +| double | float | Compatible | | +| double | decimal | Incompatible | | +| double | string | Incompatible | | +| double | timestamp | Unsupported | | +| decimal | boolean | Unsupported | | +| decimal | byte | Unsupported | | +| decimal | short | Unsupported | | +| decimal | integer | Unsupported | | +| decimal | long | Unsupported | | +| decimal | float | Compatible | | +| decimal | double | Compatible | | +| decimal | string | Unsupported | | +| decimal | timestamp | Unsupported | | +| string | boolean | Compatible | | +| string | byte | Compatible | | +| string | short | Compatible | | +| string | integer | Compatible | | +| string | long | Compatible | | +| string | float | Unsupported | | +| string | double | Unsupported | | +| string | decimal | Unsupported | | +| string | binary | Compatible | | +| string | date | Unsupported | | +| string | timestamp | Incompatible | Not all valid formats are supported | +| binary | string | Incompatible | | +| date | boolean | Unsupported | | +| date | byte | Unsupported | | +| date | short | Unsupported | | +| date | integer | Unsupported | | +| date | long | Unsupported | | +| date | float | Unsupported | | +| date | double | Unsupported | | +| date | decimal | Unsupported | | +| date | string | Compatible | | +| date | timestamp | Unsupported | | +| timestamp | boolean | Unsupported | | +| timestamp | byte | Unsupported | | +| timestamp | short | Unsupported | | +| timestamp | integer | Unsupported | | +| timestamp | long | Compatible | | +| timestamp | float | Unsupported | | +| timestamp | double | Unsupported | | +| timestamp | decimal | Unsupported | | +| timestamp | string | Compatible | | +| timestamp | date | Compatible | | From b67d57115d9ae0abacd7bfd2342444dcb8209c0c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 16:13:30 -0600 Subject: [PATCH 27/38] change default to false --- .../scala/org/apache/comet/CometConf.scala | 3 +-- .../org/apache/comet/CometCastSuite.scala | 20 ++++++++++++------- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index ec809bde4..26114090f 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -381,8 +381,7 @@ object CometConf { "Set this config to true to allow them anyway. See compatibility guide " + "for more information.") .booleanConf - // TODO change this to false and set this config explicitly in tests where needed - .createWithDefault(true) + .createWithDefault(false) } diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 267e03659..5c961f8af 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -526,8 +526,19 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } test("cast StringType to TimestampType disabled by default") { - val values = Seq("2020-01-01T12:34:56.123456", "T2").toDF("a") - castFallbackTest(values.toDF("a"), DataTypes.TimestampType, "Unsupported timezone") + withSQLConf((SQLConf.SESSION_LOCAL_TIMEZONE.key, "UTC")) { + val values = Seq("2020-01-01T12:34:56.123456", "T2").toDF("a") + castFallbackTest(values.toDF("a"), DataTypes.TimestampType, + "Not all valid formats are supported") + } + } + + test("cast StringType to TimestampType disabled for non-UTC timezone") { + withSQLConf((SQLConf.SESSION_LOCAL_TIMEZONE.key, "America/Denver")) { + val values = Seq("2020-01-01T12:34:56.123456", "T2").toDF("a") + castFallbackTest(values.toDF("a"), DataTypes.TimestampType, + "Cast will use UTC instead of Some(America/Denver)") + } } ignore("cast StringType to TimestampType (fuzz test)") { @@ -564,11 +575,6 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - test("cast StringType to TimestampType with invalid timezone") { - val values = Seq("2020-01-01T12:34:56.123456", "T2") - castFallbackTestTimezone(values.toDF("a"), DataTypes.TimestampType, "Unsupported timezone") - } - // CAST from BinaryType ignore("cast BinaryType to StringType") { From 83a70b7c26d14f748cbba45558e1d3f9070cb14f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 16:16:49 -0600 Subject: [PATCH 28/38] formatting --- .../src/test/scala/org/apache/comet/CometCastSuite.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 5c961f8af..14ea5fa53 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -528,7 +528,9 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("cast StringType to TimestampType disabled by default") { withSQLConf((SQLConf.SESSION_LOCAL_TIMEZONE.key, "UTC")) { val values = Seq("2020-01-01T12:34:56.123456", "T2").toDF("a") - castFallbackTest(values.toDF("a"), DataTypes.TimestampType, + castFallbackTest( + values.toDF("a"), + DataTypes.TimestampType, "Not all valid formats are supported") } } @@ -536,7 +538,9 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("cast StringType to TimestampType disabled for non-UTC timezone") { withSQLConf((SQLConf.SESSION_LOCAL_TIMEZONE.key, "America/Denver")) { val values = Seq("2020-01-01T12:34:56.123456", "T2").toDF("a") - castFallbackTest(values.toDF("a"), DataTypes.TimestampType, + castFallbackTest( + values.toDF("a"), + DataTypes.TimestampType, "Cast will use UTC instead of Some(America/Denver)") } } From e5226f06c2f6d6ac717e270d688c7906388f2351 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 16:58:39 -0600 Subject: [PATCH 29/38] set COMET_CAST_ALLOW_INCOMPATIBLE=true in some tests --- .../apache/comet/CometExpressionSuite.scala | 256 ++++++++++-------- .../comet/exec/CometAggregateSuite.scala | 3 +- .../apache/comet/exec/CometExecSuite.scala | 5 +- .../sql/comet/CometPlanStabilitySuite.scala | 1 + 4 files changed, 149 insertions(+), 116 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 3683c8d44..12b05c759 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -259,7 +259,9 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } test("cast timestamp and timestamp_ntz") { - withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu") { + withSQLConf( + SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu", + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { Seq(true, false).foreach { dictionaryEnabled => withTempDir { dir => val path = new Path(dir.toURI.toString, "timestamp_trunc.parquet") @@ -282,7 +284,9 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { // TODO: make the test pass for Spark 3.2 & 3.3 assume(isSpark34Plus) - withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu") { + withSQLConf( + SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu", + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { Seq(true, false).foreach { dictionaryEnabled => withTempDir { dir => val path = new Path(dir.toURI.toString, "timestamp_trunc.parquet") @@ -305,7 +309,9 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { // TODO: make the test pass for Spark 3.2 & 3.3 assume(isSpark34Plus) - withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu") { + withSQLConf( + SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu", + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { Seq(true, false).foreach { dictionaryEnabled => withTempDir { dir => val path = new Path(dir.toURI.toString, "timestamp_trunc.parquet") @@ -394,32 +400,34 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("date_trunc with timestamp_ntz") { assume(!isSpark32, "timestamp functions for timestamp_ntz have incorrect behavior in 3.2") - Seq(true, false).foreach { dictionaryEnabled => - withTempDir { dir => - val path = new Path(dir.toURI.toString, "timestamp_trunc.parquet") - makeRawTimeParquetFile(path, dictionaryEnabled = dictionaryEnabled, 10000) - withParquetTable(path.toString, "timetbl") { - Seq( - "YEAR", - "YYYY", - "YY", - "MON", - "MONTH", - "MM", - "QUARTER", - "WEEK", - "DAY", - "DD", - "HOUR", - "MINUTE", - "SECOND", - "MILLISECOND", - "MICROSECOND").foreach { format => - checkSparkAnswerAndOperator( - "SELECT " + - s"date_trunc('$format', _3), " + - s"date_trunc('$format', _5) " + - " from timetbl") + withSQLConf(CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "timestamp_trunc.parquet") + makeRawTimeParquetFile(path, dictionaryEnabled = dictionaryEnabled, 10000) + withParquetTable(path.toString, "timetbl") { + Seq( + "YEAR", + "YYYY", + "YY", + "MON", + "MONTH", + "MM", + "QUARTER", + "WEEK", + "DAY", + "DD", + "HOUR", + "MINUTE", + "SECOND", + "MILLISECOND", + "MICROSECOND").foreach { format => + checkSparkAnswerAndOperator( + "SELECT " + + s"date_trunc('$format', _3), " + + s"date_trunc('$format', _5) " + + " from timetbl") + } } } } @@ -428,22 +436,24 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("date_trunc with format array") { assume(isSpark33Plus, "TimestampNTZ is supported in Spark 3.3+, See SPARK-36182") - val numRows = 1000 - Seq(true, false).foreach { dictionaryEnabled => - withTempDir { dir => - val path = new Path(dir.toURI.toString, "timestamp_trunc_with_format.parquet") - makeDateTimeWithFormatTable(path, dictionaryEnabled = dictionaryEnabled, numRows) - withParquetTable(path.toString, "timeformattbl") { - checkSparkAnswerAndOperator( - "SELECT " + - "format, _0, _1, _2, _3, _4, _5, " + - "date_trunc(format, _0), " + - "date_trunc(format, _1), " + - "date_trunc(format, _2), " + - "date_trunc(format, _3), " + - "date_trunc(format, _4), " + - "date_trunc(format, _5) " + - " from timeformattbl ") + withSQLConf(CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + val numRows = 1000 + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "timestamp_trunc_with_format.parquet") + makeDateTimeWithFormatTable(path, dictionaryEnabled = dictionaryEnabled, numRows) + withParquetTable(path.toString, "timeformattbl") { + checkSparkAnswerAndOperator( + "SELECT " + + "format, _0, _1, _2, _3, _4, _5, " + + "date_trunc(format, _0), " + + "date_trunc(format, _1), " + + "date_trunc(format, _2), " + + "date_trunc(format, _3), " + + "date_trunc(format, _4), " + + "date_trunc(format, _5) " + + " from timeformattbl ") + } } } } @@ -552,83 +562,88 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("like (LikeSimplification enabled)") { val table = "names" - withTable(table) { - sql(s"create table $table(id int, name varchar(20)) using parquet") - sql(s"insert into $table values(1,'James Smith')") - sql(s"insert into $table values(2,'Michael Rose')") - sql(s"insert into $table values(3,'Robert Williams')") - sql(s"insert into $table values(4,'Rames Rose')") - sql(s"insert into $table values(5,'Rames rose')") - - // Filter column having values 'Rames _ose', where any character matches for '_' - val query = sql(s"select id from $table where name like 'Rames _ose'") - checkAnswer(query, Row(4) :: Row(5) :: Nil) - - // Filter rows that contains 'rose' in 'name' column - val queryContains = sql(s"select id from $table where name like '%rose%'") - checkAnswer(queryContains, Row(5) :: Nil) - - // Filter rows that starts with 'R' following by any characters - val queryStartsWith = sql(s"select id from $table where name like 'R%'") - checkAnswer(queryStartsWith, Row(3) :: Row(4) :: Row(5) :: Nil) - - // Filter rows that ends with 's' following by any characters - val queryEndsWith = sql(s"select id from $table where name like '%s'") - checkAnswer(queryEndsWith, Row(3) :: Nil) + withSQLConf(CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter column having values 'Rames _ose', where any character matches for '_' + val query = sql(s"select id from $table where name like 'Rames _ose'") + checkAnswer(query, Row(4) :: Row(5) :: Nil) + + // Filter rows that contains 'rose' in 'name' column + val queryContains = sql(s"select id from $table where name like '%rose%'") + checkAnswer(queryContains, Row(5) :: Nil) + + // Filter rows that starts with 'R' following by any characters + val queryStartsWith = sql(s"select id from $table where name like 'R%'") + checkAnswer(queryStartsWith, Row(3) :: Row(4) :: Row(5) :: Nil) + + // Filter rows that ends with 's' following by any characters + val queryEndsWith = sql(s"select id from $table where name like '%s'") + checkAnswer(queryEndsWith, Row(3) :: Nil) + } } } test("contains") { assume(!isSpark32) - - val table = "names" - withTable(table) { - sql(s"create table $table(id int, name varchar(20)) using parquet") - sql(s"insert into $table values(1,'James Smith')") - sql(s"insert into $table values(2,'Michael Rose')") - sql(s"insert into $table values(3,'Robert Williams')") - sql(s"insert into $table values(4,'Rames Rose')") - sql(s"insert into $table values(5,'Rames rose')") - - // Filter rows that contains 'rose' in 'name' column - val queryContains = sql(s"select id from $table where contains (name, 'rose')") - checkAnswer(queryContains, Row(5) :: Nil) + withSQLConf(CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter rows that contains 'rose' in 'name' column + val queryContains = sql(s"select id from $table where contains (name, 'rose')") + checkAnswer(queryContains, Row(5) :: Nil) + } } } test("startswith") { assume(!isSpark32) - - val table = "names" - withTable(table) { - sql(s"create table $table(id int, name varchar(20)) using parquet") - sql(s"insert into $table values(1,'James Smith')") - sql(s"insert into $table values(2,'Michael Rose')") - sql(s"insert into $table values(3,'Robert Williams')") - sql(s"insert into $table values(4,'Rames Rose')") - sql(s"insert into $table values(5,'Rames rose')") - - // Filter rows that starts with 'R' following by any characters - val queryStartsWith = sql(s"select id from $table where startswith (name, 'R')") - checkAnswer(queryStartsWith, Row(3) :: Row(4) :: Row(5) :: Nil) + withSQLConf(CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter rows that starts with 'R' following by any characters + val queryStartsWith = sql(s"select id from $table where startswith (name, 'R')") + checkAnswer(queryStartsWith, Row(3) :: Row(4) :: Row(5) :: Nil) + } } } test("endswith") { assume(!isSpark32) - - val table = "names" - withTable(table) { - sql(s"create table $table(id int, name varchar(20)) using parquet") - sql(s"insert into $table values(1,'James Smith')") - sql(s"insert into $table values(2,'Michael Rose')") - sql(s"insert into $table values(3,'Robert Williams')") - sql(s"insert into $table values(4,'Rames Rose')") - sql(s"insert into $table values(5,'Rames rose')") - - // Filter rows that ends with 's' following by any characters - val queryEndsWith = sql(s"select id from $table where endswith (name, 's')") - checkAnswer(queryEndsWith, Row(3) :: Nil) + withSQLConf(CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter rows that ends with 's' following by any characters + val queryEndsWith = sql(s"select id from $table where endswith (name, 's')") + checkAnswer(queryEndsWith, Row(3) :: Nil) + } } } @@ -818,7 +833,9 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("ceil and floor") { Seq("true", "false").foreach { dictionary => - withSQLConf("parquet.enable.dictionary" -> dictionary) { + withSQLConf( + "parquet.enable.dictionary" -> dictionary, + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { withParquetTable( (-5 until 5).map(i => (i.toDouble + 0.3, i.toDouble + 0.8)), "tbl", @@ -914,7 +931,9 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("Various String scalar functions") { Seq(false, true).foreach { dictionary => - withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + withSQLConf( + "parquet.enable.dictionary" -> dictionary.toString, + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { val table = "names" withTable(table) { sql(s"create table $table(id int, name varchar(20)) using parquet") @@ -944,7 +963,9 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("InitCap") { Seq(false, true).foreach { dictionary => - withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + withSQLConf( + "parquet.enable.dictionary" -> dictionary.toString, + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { val table = "names" withTable(table) { sql(s"create table $table(id int, name varchar(20)) using parquet") @@ -997,7 +1018,9 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("string concat_ws") { Seq(false, true).foreach { dictionary => - withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + withSQLConf( + "parquet.enable.dictionary" -> dictionary.toString, + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { val table = "names" withTable(table) { sql( @@ -1014,7 +1037,9 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("string repeat") { Seq(false, true).foreach { dictionary => - withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + withSQLConf( + "parquet.enable.dictionary" -> dictionary.toString, + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { val table = "names" withTable(table) { sql(s"create table $table(id int, name varchar(20)) using parquet") @@ -1094,7 +1119,8 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { Seq(false, true).foreach { dictionary => withSQLConf( SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> inSetThreshold, - "parquet.enable.dictionary" -> dictionary.toString) { + "parquet.enable.dictionary" -> dictionary.toString, + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { val table = "names" withTable(table) { sql(s"create table $table(id int, name varchar(20)) using parquet") @@ -1406,7 +1432,9 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("hash functions") { Seq(true, false).foreach { dictionary => - withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + 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") 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 bd4042ec1..fc6876fd1 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala @@ -863,7 +863,8 @@ 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_COLUMNAR_SHUFFLE_ENABLED.key -> "false", + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { withTempDir { dir => val path = new Path(dir.toURI.toString, "test") makeParquetFile(path, 1000, 20, dictionaryEnabled) 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 f37183563..cd77ae94b 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -884,6 +884,7 @@ class CometExecSuite extends CometTestBase { test("like (LikeSimplification disabled)") { val table = "names" withSQLConf( + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true", SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> "org.apache.spark.sql.catalyst.optimizer.LikeSimplification") { withTable(table) { sql(s"create table $table(id int, name varchar(20)) using parquet") @@ -949,7 +950,9 @@ class CometExecSuite extends CometTestBase { } test("SPARK-33474: Support typed literals as partition spec values") { - withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu") { + withSQLConf( + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true", + SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu") { withTable("t1") { val binaryStr = "Spark SQL" val binaryHexStr = Hex.hex(UTF8String.fromString(binaryStr).getBytes).toString diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala index 48969ea41..90ea79473 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala @@ -261,6 +261,7 @@ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBa CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", CometConf.COMET_EXEC_ALL_EXPR_ENABLED.key -> "true", + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true", // needed for v1.4/q9, v1.4/q44, v2.7.0/q6, v2.7.0/q64 "spark.sql.readSideCharPadding" -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10MB") { val qe = sql(queryString).queryExecution From 67385442082199fb30d930b96b3f39329ee4c5c3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 17:58:49 -0600 Subject: [PATCH 30/38] revert a change --- .../test/scala/org/apache/comet/exec/CometExecSuite.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) 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 cd77ae94b..319f87d6c 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -884,7 +884,6 @@ class CometExecSuite extends CometTestBase { test("like (LikeSimplification disabled)") { val table = "names" withSQLConf( - CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true", SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> "org.apache.spark.sql.catalyst.optimizer.LikeSimplification") { withTable(table) { sql(s"create table $table(id int, name varchar(20)) using parquet") @@ -950,9 +949,8 @@ class CometExecSuite extends CometTestBase { } test("SPARK-33474: Support typed literals as partition spec values") { - withSQLConf( - CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true", - SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu") { + withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu", + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { withTable("t1") { val binaryStr = "Spark SQL" val binaryHexStr = Hex.hex(UTF8String.fromString(binaryStr).getBytes).toString From a1bfdee5723937c6b8ceb95b2a08745fa1e0e809 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 18:02:34 -0600 Subject: [PATCH 31/38] spark 3.2 does not support cast timestamp to string --- spark/src/test/scala/org/apache/comet/CometCastSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 14ea5fa53..5a55723c8 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -662,6 +662,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } test("cast TimestampType to LongType") { + assume(CometSparkSessionExtensions.isSpark33Plus) castTest(generateTimestamps(), DataTypes.LongType) } From 15c4989c8ae3bff2093e41169096d1bef7fe7452 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 May 2024 18:10:19 -0600 Subject: [PATCH 32/38] spotless --- .../test/scala/org/apache/comet/exec/CometExecSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 319f87d6c..bfde14033 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -949,8 +949,9 @@ class CometExecSuite extends CometTestBase { } test("SPARK-33474: Support typed literals as partition spec values") { - withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu", - CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + withSQLConf( + SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu", + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { withTable("t1") { val binaryStr = "Spark SQL" val binaryHexStr = Hex.hex(UTF8String.fromString(binaryStr).getBytes).toString From 8c40e177eb7a395dd44790305664ccec296c42ee Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 May 2024 08:24:18 -0600 Subject: [PATCH 33/38] add links to issues --- .../scala/org/apache/comet/expressions/CometCast.scala | 10 ++++++---- .../test/scala/org/apache/comet/CometCastSuite.scala | 3 ++- 2 files changed, 8 insertions(+), 5 deletions(-) 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 d37bcd8e7..5641c94a8 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -49,6 +49,7 @@ object CometCast { DataTypes.DateType, DataTypes.TimestampType) // TODO add DataTypes.TimestampNTZType for Spark 3.4 and later + // https://github.com/apache/datafusion-comet/issues/378 def isSupported( fromType: DataType, @@ -62,17 +63,15 @@ object CometCast { (fromType, toType) match { case (dt: DataType, _) if dt.typeName == "timestamp_ntz" => + // https://github.com/apache/datafusion-comet/issues/378 toType match { case DataTypes.TimestampType | DataTypes.DateType | DataTypes.StringType => Incompatible() case _ => Unsupported } - case (DataTypes.BinaryType, DataTypes.StringType) => - Incompatible() case (_: DecimalType, _: DecimalType) => - // TODO we need to file an issue for adding specific tests for casting - // between decimal types with different precision and scale + // https://github.com/apache/datafusion-comet/issues/375 Incompatible() case (DataTypes.StringType, _) => canCastFromString(toType, timeZoneId, evalMode) @@ -140,6 +139,9 @@ object CometCast { case DataTypes.FloatType | DataTypes.DoubleType => // https://github.com/apache/datafusion-comet/issues/326 Incompatible() + case DataTypes.BinaryType => + // https://github.com/apache/datafusion-comet/issues/377 + Incompatible() case _ => Unsupported } } diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 5a55723c8..3824413de 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -582,7 +582,8 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { // CAST from BinaryType ignore("cast BinaryType to StringType") { - // TODO + // TODO implement this + // https://github.com/apache/datafusion-comet/issues/377 } // CAST from DateType From be9449567a65fc109103f7046dac87c980476ac0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 May 2024 08:28:37 -0600 Subject: [PATCH 34/38] regenerate config docs --- docs/source/user-guide/configs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 570723626..02ecbd693 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -25,7 +25,7 @@ Comet provides the following configuration settings. |--------|-------------|---------------| | spark.comet.ansi.enabled | Comet does not respect ANSI mode in most cases and by default will not accelerate queries when ansi mode is enabled. Enable this setting to test Comet's experimental support for ANSI mode. This should not be used in production. | false | | spark.comet.batchSize | The columnar batch size, i.e., the maximum number of rows that a batch can contain. | 8192 | -| spark.comet.cast.allowIncompatible | Comet is not currently fully compatible with Spark for all cast operations. Set this config to true to allow them anyway. See compatibility guide for more information. | true | +| spark.comet.cast.allowIncompatible | Comet is not currently fully compatible with Spark for all cast operations. Set this config to true to allow them anyway. See compatibility guide for more information. | false | | 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 | From 581d92e040af3bba5ac13c589a26e284fdd21762 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 May 2024 08:29:24 -0600 Subject: [PATCH 35/38] prettier --- docs/source/user-guide/configs.md | 58 +++++++++++++++---------------- 1 file changed, 29 insertions(+), 29 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 02ecbd693..aef96e91b 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -21,32 +21,32 @@ under the License. Comet provides the following configuration settings. -| Config | Description | Default Value | -|--------|-------------|---------------| -| spark.comet.ansi.enabled | Comet does not respect ANSI mode in most cases and by default will not accelerate queries when ansi mode is enabled. Enable this setting to test Comet's experimental support for ANSI mode. This should not be used in production. | false | -| spark.comet.batchSize | The columnar batch size, i.e., the maximum number of rows that a batch can contain. | 8192 | -| spark.comet.cast.allowIncompatible | Comet is not currently fully compatible with Spark for all cast operations. Set this config to true to allow them anyway. See compatibility guide for more information. | false | -| 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 | Force Comet to only use columnar shuffle for CometScan and Spark regular operators. If this is enabled, Comet native shuffle will not be enabled but only Arrow shuffle. By default, this config is false. | false | -| 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 | -| spark.comet.exceptionOnDatetimeRebase | Whether to throw exception when seeing dates/timestamps from the legacy hybrid (Julian + Gregorian) calendar. Since Spark 3, dates/timestamps were written according to the Proleptic Gregorian calendar. When this is true, Comet will throw exceptions when seeing these dates/timestamps that were written by Spark version before 3.0. If this is false, these dates/timestamps will be read as if they were written to the Proleptic Gregorian calendar and will not be rebased. | false | -| spark.comet.exec.all.enabled | Whether to enable all Comet operators. By default, this config is false. Note that this config precedes all separate config 'spark.comet.exec..enabled'. That being said, if this config is enabled, separate configs are ignored. | false | -| spark.comet.exec.all.expr.enabled | Whether to enable all Comet exprs. By default, this config is false. Note that this config precedes all separate config 'spark.comet.exec..enabled'. That being said, if this config is enabled, separate configs are ignored. | false | -| spark.comet.exec.broadcast.enabled | Whether to force enabling broadcasting for Comet native operators. By default, this config is false. Comet broadcast feature will be enabled automatically by Comet extension. But for unit tests, we need this feature to force enabling it for invalid cases. So this config is only used for unit test. | false | -| spark.comet.exec.enabled | Whether to enable Comet native vectorized execution for Spark. This controls whether Spark should convert operators into their Comet counterparts and execute them in native space. Note: each operator is associated with a separate config in the format of 'spark.comet.exec..enabled' at the moment, and both the config and this need to be turned on, in order for the operator to be executed in native. By default, this config is false. | false | -| 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.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 | -| spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false | -| spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. By default, this is false | false | -| spark.comet.rowToColumnar.supportedOperatorList | A comma-separated list of row-based operators that will be converted to columnar format when 'spark.comet.rowToColumnar.enabled' is true | Range,InMemoryTableScan | -| spark.comet.scan.enabled | Whether to enable Comet scan. 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 true. | true | -| spark.comet.scan.preFetch.enabled | Whether to enable pre-fetching feature of CometScan. By default is disabled. | false | -| spark.comet.scan.preFetch.threadNum | The number of threads running pre-fetching for CometScan. Effective if spark.comet.scan.preFetch.enabled is enabled. By default it is 2. Note that more pre-fetching threads means more memory requirement to store pre-fetched row groups. | 2 | -| spark.comet.shuffle.preferDictionary.ratio | The ratio of total values to distinct values in a string column to decide whether to prefer dictionary encoding when shuffling the column. If the ratio is higher than this config, dictionary encoding will be used on shuffling string column. This config is effective if it is higher than 1.0. By default, this config is 10.0. Note that this config is only used when 'spark.comet.columnar.shuffle.enabled' is true. | 10.0 | +| Config | Description | Default Value | +| ------------------------------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ----------------------- | +| spark.comet.ansi.enabled | Comet does not respect ANSI mode in most cases and by default will not accelerate queries when ansi mode is enabled. Enable this setting to test Comet's experimental support for ANSI mode. This should not be used in production. | false | +| spark.comet.batchSize | The columnar batch size, i.e., the maximum number of rows that a batch can contain. | 8192 | +| spark.comet.cast.allowIncompatible | Comet is not currently fully compatible with Spark for all cast operations. Set this config to true to allow them anyway. See compatibility guide for more information. | false | +| 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 | Force Comet to only use columnar shuffle for CometScan and Spark regular operators. If this is enabled, Comet native shuffle will not be enabled but only Arrow shuffle. By default, this config is false. | false | +| 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 | +| spark.comet.exceptionOnDatetimeRebase | Whether to throw exception when seeing dates/timestamps from the legacy hybrid (Julian + Gregorian) calendar. Since Spark 3, dates/timestamps were written according to the Proleptic Gregorian calendar. When this is true, Comet will throw exceptions when seeing these dates/timestamps that were written by Spark version before 3.0. If this is false, these dates/timestamps will be read as if they were written to the Proleptic Gregorian calendar and will not be rebased. | false | +| spark.comet.exec.all.enabled | Whether to enable all Comet operators. By default, this config is false. Note that this config precedes all separate config 'spark.comet.exec..enabled'. That being said, if this config is enabled, separate configs are ignored. | false | +| spark.comet.exec.all.expr.enabled | Whether to enable all Comet exprs. By default, this config is false. Note that this config precedes all separate config 'spark.comet.exec..enabled'. That being said, if this config is enabled, separate configs are ignored. | false | +| spark.comet.exec.broadcast.enabled | Whether to force enabling broadcasting for Comet native operators. By default, this config is false. Comet broadcast feature will be enabled automatically by Comet extension. But for unit tests, we need this feature to force enabling it for invalid cases. So this config is only used for unit test. | false | +| spark.comet.exec.enabled | Whether to enable Comet native vectorized execution for Spark. This controls whether Spark should convert operators into their Comet counterparts and execute them in native space. Note: each operator is associated with a separate config in the format of 'spark.comet.exec..enabled' at the moment, and both the config and this need to be turned on, in order for the operator to be executed in native. By default, this config is false. | false | +| 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.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 | +| spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false | +| spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. By default, this is false | false | +| spark.comet.rowToColumnar.supportedOperatorList | A comma-separated list of row-based operators that will be converted to columnar format when 'spark.comet.rowToColumnar.enabled' is true | Range,InMemoryTableScan | +| spark.comet.scan.enabled | Whether to enable Comet scan. 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 true. | true | +| spark.comet.scan.preFetch.enabled | Whether to enable pre-fetching feature of CometScan. By default is disabled. | false | +| spark.comet.scan.preFetch.threadNum | The number of threads running pre-fetching for CometScan. Effective if spark.comet.scan.preFetch.enabled is enabled. By default it is 2. Note that more pre-fetching threads means more memory requirement to store pre-fetched row groups. | 2 | +| spark.comet.shuffle.preferDictionary.ratio | The ratio of total values to distinct values in a string column to decide whether to prefer dictionary encoding when shuffling the column. If the ratio is higher than this config, dictionary encoding will be used on shuffling string column. This config is effective if it is higher than 1.0. By default, this config is 10.0. Note that this config is only used when 'spark.comet.columnar.shuffle.enabled' is true. | 10.0 | From 4374fc0dfcbebce65240ddc4304ba33b37cc51a0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 May 2024 08:29:57 -0600 Subject: [PATCH 36/38] revert prettier --- docs/source/user-guide/configs.md | 58 +++++++++++++++---------------- 1 file changed, 29 insertions(+), 29 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index aef96e91b..02ecbd693 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -21,32 +21,32 @@ under the License. Comet provides the following configuration settings. -| Config | Description | Default Value | -| ------------------------------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ----------------------- | -| spark.comet.ansi.enabled | Comet does not respect ANSI mode in most cases and by default will not accelerate queries when ansi mode is enabled. Enable this setting to test Comet's experimental support for ANSI mode. This should not be used in production. | false | -| spark.comet.batchSize | The columnar batch size, i.e., the maximum number of rows that a batch can contain. | 8192 | -| spark.comet.cast.allowIncompatible | Comet is not currently fully compatible with Spark for all cast operations. Set this config to true to allow them anyway. See compatibility guide for more information. | false | -| 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 | Force Comet to only use columnar shuffle for CometScan and Spark regular operators. If this is enabled, Comet native shuffle will not be enabled but only Arrow shuffle. By default, this config is false. | false | -| 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 | -| spark.comet.exceptionOnDatetimeRebase | Whether to throw exception when seeing dates/timestamps from the legacy hybrid (Julian + Gregorian) calendar. Since Spark 3, dates/timestamps were written according to the Proleptic Gregorian calendar. When this is true, Comet will throw exceptions when seeing these dates/timestamps that were written by Spark version before 3.0. If this is false, these dates/timestamps will be read as if they were written to the Proleptic Gregorian calendar and will not be rebased. | false | -| spark.comet.exec.all.enabled | Whether to enable all Comet operators. By default, this config is false. Note that this config precedes all separate config 'spark.comet.exec..enabled'. That being said, if this config is enabled, separate configs are ignored. | false | -| spark.comet.exec.all.expr.enabled | Whether to enable all Comet exprs. By default, this config is false. Note that this config precedes all separate config 'spark.comet.exec..enabled'. That being said, if this config is enabled, separate configs are ignored. | false | -| spark.comet.exec.broadcast.enabled | Whether to force enabling broadcasting for Comet native operators. By default, this config is false. Comet broadcast feature will be enabled automatically by Comet extension. But for unit tests, we need this feature to force enabling it for invalid cases. So this config is only used for unit test. | false | -| spark.comet.exec.enabled | Whether to enable Comet native vectorized execution for Spark. This controls whether Spark should convert operators into their Comet counterparts and execute them in native space. Note: each operator is associated with a separate config in the format of 'spark.comet.exec..enabled' at the moment, and both the config and this need to be turned on, in order for the operator to be executed in native. By default, this config is false. | false | -| 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.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 | -| spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false | -| spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. By default, this is false | false | -| spark.comet.rowToColumnar.supportedOperatorList | A comma-separated list of row-based operators that will be converted to columnar format when 'spark.comet.rowToColumnar.enabled' is true | Range,InMemoryTableScan | -| spark.comet.scan.enabled | Whether to enable Comet scan. 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 true. | true | -| spark.comet.scan.preFetch.enabled | Whether to enable pre-fetching feature of CometScan. By default is disabled. | false | -| spark.comet.scan.preFetch.threadNum | The number of threads running pre-fetching for CometScan. Effective if spark.comet.scan.preFetch.enabled is enabled. By default it is 2. Note that more pre-fetching threads means more memory requirement to store pre-fetched row groups. | 2 | -| spark.comet.shuffle.preferDictionary.ratio | The ratio of total values to distinct values in a string column to decide whether to prefer dictionary encoding when shuffling the column. If the ratio is higher than this config, dictionary encoding will be used on shuffling string column. This config is effective if it is higher than 1.0. By default, this config is 10.0. Note that this config is only used when 'spark.comet.columnar.shuffle.enabled' is true. | 10.0 | +| Config | Description | Default Value | +|--------|-------------|---------------| +| spark.comet.ansi.enabled | Comet does not respect ANSI mode in most cases and by default will not accelerate queries when ansi mode is enabled. Enable this setting to test Comet's experimental support for ANSI mode. This should not be used in production. | false | +| spark.comet.batchSize | The columnar batch size, i.e., the maximum number of rows that a batch can contain. | 8192 | +| spark.comet.cast.allowIncompatible | Comet is not currently fully compatible with Spark for all cast operations. Set this config to true to allow them anyway. See compatibility guide for more information. | false | +| 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 | Force Comet to only use columnar shuffle for CometScan and Spark regular operators. If this is enabled, Comet native shuffle will not be enabled but only Arrow shuffle. By default, this config is false. | false | +| 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 | +| spark.comet.exceptionOnDatetimeRebase | Whether to throw exception when seeing dates/timestamps from the legacy hybrid (Julian + Gregorian) calendar. Since Spark 3, dates/timestamps were written according to the Proleptic Gregorian calendar. When this is true, Comet will throw exceptions when seeing these dates/timestamps that were written by Spark version before 3.0. If this is false, these dates/timestamps will be read as if they were written to the Proleptic Gregorian calendar and will not be rebased. | false | +| spark.comet.exec.all.enabled | Whether to enable all Comet operators. By default, this config is false. Note that this config precedes all separate config 'spark.comet.exec..enabled'. That being said, if this config is enabled, separate configs are ignored. | false | +| spark.comet.exec.all.expr.enabled | Whether to enable all Comet exprs. By default, this config is false. Note that this config precedes all separate config 'spark.comet.exec..enabled'. That being said, if this config is enabled, separate configs are ignored. | false | +| spark.comet.exec.broadcast.enabled | Whether to force enabling broadcasting for Comet native operators. By default, this config is false. Comet broadcast feature will be enabled automatically by Comet extension. But for unit tests, we need this feature to force enabling it for invalid cases. So this config is only used for unit test. | false | +| spark.comet.exec.enabled | Whether to enable Comet native vectorized execution for Spark. This controls whether Spark should convert operators into their Comet counterparts and execute them in native space. Note: each operator is associated with a separate config in the format of 'spark.comet.exec..enabled' at the moment, and both the config and this need to be turned on, in order for the operator to be executed in native. By default, this config is false. | false | +| 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.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 | +| spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false | +| spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. By default, this is false | false | +| spark.comet.rowToColumnar.supportedOperatorList | A comma-separated list of row-based operators that will be converted to columnar format when 'spark.comet.rowToColumnar.enabled' is true | Range,InMemoryTableScan | +| spark.comet.scan.enabled | Whether to enable Comet scan. 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 true. | true | +| spark.comet.scan.preFetch.enabled | Whether to enable pre-fetching feature of CometScan. By default is disabled. | false | +| spark.comet.scan.preFetch.threadNum | The number of threads running pre-fetching for CometScan. Effective if spark.comet.scan.preFetch.enabled is enabled. By default it is 2. Note that more pre-fetching threads means more memory requirement to store pre-fetched row groups. | 2 | +| spark.comet.shuffle.preferDictionary.ratio | The ratio of total values to distinct values in a string column to decide whether to prefer dictionary encoding when shuffling the column. If the ratio is higher than this config, dictionary encoding will be used on shuffling string column. This config is effective if it is higher than 1.0. By default, this config is 10.0. Note that this config is only used when 'spark.comet.columnar.shuffle.enabled' is true. | 10.0 | From 2e5e33beb781eea3d99fc38afbddc1e4acd692ae Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 May 2024 08:41:53 -0600 Subject: [PATCH 37/38] revert a change --- spark/src/test/scala/org/apache/comet/CometCastSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 3824413de..de0e951ba 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -67,6 +67,8 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } else if (!testExists) { fail(s"Missing test: $expectedTestName") } + } else if (testExists) { + fail(s"Found test for cast that Spark does not support: $expectedTestName") } } } From 128e763b7e86042be0f88ccbddc2f3748c3b59aa Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 May 2024 11:59:54 -0600 Subject: [PATCH 38/38] revert some redundant test changes --- .../apache/comet/CometExpressionSuite.scala | 148 ++++++++---------- 1 file changed, 67 insertions(+), 81 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 12b05c759..c8c7ffd5c 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -562,88 +562,83 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("like (LikeSimplification enabled)") { val table = "names" - withSQLConf(CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { - withTable(table) { - sql(s"create table $table(id int, name varchar(20)) using parquet") - sql(s"insert into $table values(1,'James Smith')") - sql(s"insert into $table values(2,'Michael Rose')") - sql(s"insert into $table values(3,'Robert Williams')") - sql(s"insert into $table values(4,'Rames Rose')") - sql(s"insert into $table values(5,'Rames rose')") - - // Filter column having values 'Rames _ose', where any character matches for '_' - val query = sql(s"select id from $table where name like 'Rames _ose'") - checkAnswer(query, Row(4) :: Row(5) :: Nil) - - // Filter rows that contains 'rose' in 'name' column - val queryContains = sql(s"select id from $table where name like '%rose%'") - checkAnswer(queryContains, Row(5) :: Nil) - - // Filter rows that starts with 'R' following by any characters - val queryStartsWith = sql(s"select id from $table where name like 'R%'") - checkAnswer(queryStartsWith, Row(3) :: Row(4) :: Row(5) :: Nil) - - // Filter rows that ends with 's' following by any characters - val queryEndsWith = sql(s"select id from $table where name like '%s'") - checkAnswer(queryEndsWith, Row(3) :: Nil) - } + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter column having values 'Rames _ose', where any character matches for '_' + val query = sql(s"select id from $table where name like 'Rames _ose'") + checkAnswer(query, Row(4) :: Row(5) :: Nil) + + // Filter rows that contains 'rose' in 'name' column + val queryContains = sql(s"select id from $table where name like '%rose%'") + checkAnswer(queryContains, Row(5) :: Nil) + + // Filter rows that starts with 'R' following by any characters + val queryStartsWith = sql(s"select id from $table where name like 'R%'") + checkAnswer(queryStartsWith, Row(3) :: Row(4) :: Row(5) :: Nil) + + // Filter rows that ends with 's' following by any characters + val queryEndsWith = sql(s"select id from $table where name like '%s'") + checkAnswer(queryEndsWith, Row(3) :: Nil) } } test("contains") { assume(!isSpark32) - withSQLConf(CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { - val table = "names" - withTable(table) { - sql(s"create table $table(id int, name varchar(20)) using parquet") - sql(s"insert into $table values(1,'James Smith')") - sql(s"insert into $table values(2,'Michael Rose')") - sql(s"insert into $table values(3,'Robert Williams')") - sql(s"insert into $table values(4,'Rames Rose')") - sql(s"insert into $table values(5,'Rames rose')") - - // Filter rows that contains 'rose' in 'name' column - val queryContains = sql(s"select id from $table where contains (name, 'rose')") - checkAnswer(queryContains, Row(5) :: Nil) - } + + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter rows that contains 'rose' in 'name' column + val queryContains = sql(s"select id from $table where contains (name, 'rose')") + checkAnswer(queryContains, Row(5) :: Nil) } } test("startswith") { assume(!isSpark32) - withSQLConf(CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { - val table = "names" - withTable(table) { - sql(s"create table $table(id int, name varchar(20)) using parquet") - sql(s"insert into $table values(1,'James Smith')") - sql(s"insert into $table values(2,'Michael Rose')") - sql(s"insert into $table values(3,'Robert Williams')") - sql(s"insert into $table values(4,'Rames Rose')") - sql(s"insert into $table values(5,'Rames rose')") - - // Filter rows that starts with 'R' following by any characters - val queryStartsWith = sql(s"select id from $table where startswith (name, 'R')") - checkAnswer(queryStartsWith, Row(3) :: Row(4) :: Row(5) :: Nil) - } + + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter rows that starts with 'R' following by any characters + val queryStartsWith = sql(s"select id from $table where startswith (name, 'R')") + checkAnswer(queryStartsWith, Row(3) :: Row(4) :: Row(5) :: Nil) } } test("endswith") { assume(!isSpark32) - withSQLConf(CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { - val table = "names" - withTable(table) { - sql(s"create table $table(id int, name varchar(20)) using parquet") - sql(s"insert into $table values(1,'James Smith')") - sql(s"insert into $table values(2,'Michael Rose')") - sql(s"insert into $table values(3,'Robert Williams')") - sql(s"insert into $table values(4,'Rames Rose')") - sql(s"insert into $table values(5,'Rames rose')") - - // Filter rows that ends with 's' following by any characters - val queryEndsWith = sql(s"select id from $table where endswith (name, 's')") - checkAnswer(queryEndsWith, Row(3) :: Nil) - } + + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter rows that ends with 's' following by any characters + val queryEndsWith = sql(s"select id from $table where endswith (name, 's')") + checkAnswer(queryEndsWith, Row(3) :: Nil) } } @@ -931,9 +926,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("Various String scalar functions") { Seq(false, true).foreach { dictionary => - withSQLConf( - "parquet.enable.dictionary" -> dictionary.toString, - CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { val table = "names" withTable(table) { sql(s"create table $table(id int, name varchar(20)) using parquet") @@ -963,9 +956,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("InitCap") { Seq(false, true).foreach { dictionary => - withSQLConf( - "parquet.enable.dictionary" -> dictionary.toString, - CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { val table = "names" withTable(table) { sql(s"create table $table(id int, name varchar(20)) using parquet") @@ -1018,9 +1009,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("string concat_ws") { Seq(false, true).foreach { dictionary => - withSQLConf( - "parquet.enable.dictionary" -> dictionary.toString, - CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { val table = "names" withTable(table) { sql( @@ -1037,9 +1026,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("string repeat") { Seq(false, true).foreach { dictionary => - withSQLConf( - "parquet.enable.dictionary" -> dictionary.toString, - CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { val table = "names" withTable(table) { sql(s"create table $table(id int, name varchar(20)) using parquet") @@ -1119,8 +1106,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { Seq(false, true).foreach { dictionary => withSQLConf( SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> inSetThreshold, - "parquet.enable.dictionary" -> dictionary.toString, - CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + "parquet.enable.dictionary" -> dictionary.toString) { val table = "names" withTable(table) { sql(s"create table $table(id int, name varchar(20)) using parquet")