From 801752ccca1b48cd6f153519f9b9646c4c104ef0 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 15 Nov 2023 16:58:34 +0800 Subject: [PATCH 01/46] Support current_date with timezone Signed-off-by: Ferdinand Xu --- .../nvidia/spark/rapids/GpuOverrides.scala | 4 +- .../com/nvidia/spark/rapids/Plugin.scala | 12 ++--- .../com/nvidia/spark/rapids/RapidsMeta.scala | 4 ++ .../com/nvidia/spark/rapids/TypeChecks.scala | 9 ++-- .../spark/rapids/ParseDateTimeSuite.scala | 50 ++++++++++++++++++- 5 files changed, 64 insertions(+), 15 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 8119e78d988..8c4a90b7195 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -669,9 +669,7 @@ object GpuOverrides extends Logging { case FloatType => true case DoubleType => true case DateType => true - case TimestampType => - TypeChecks.areTimestampsSupported(ZoneId.systemDefault()) && - TypeChecks.areTimestampsSupported(SQLConf.get.sessionLocalTimeZone) + case TimestampType => true case StringType => true case dt: DecimalType if allowDecimal => dt.precision <= DType.DECIMAL64_MAX_PRECISION case NullType => allowNull diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala index 6520ff4c1b7..cf92dce30dc 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala @@ -374,13 +374,11 @@ class RapidsExecutorPlugin extends ExecutorPlugin with Logging { case Some(value) => ZoneId.of(value) case None => throw new RuntimeException(s"Driver time zone cannot be determined.") } - if (TypeChecks.areTimestampsSupported(driverTimezone)) { - val executorTimezone = ZoneId.systemDefault() - if (executorTimezone.normalized() != driverTimezone.normalized()) { - throw new RuntimeException(s" Driver and executor timezone mismatch. " + - s"Driver timezone is $driverTimezone and executor timezone is " + - s"$executorTimezone. Set executor timezone to $driverTimezone.") - } + val executorTimezone = ZoneId.systemDefault() + if (executorTimezone.normalized() != driverTimezone.normalized()) { + throw new RuntimeException(s" Driver and executor timezone mismatch. " + + s"Driver timezone is $driverTimezone and executor timezone is " + + s"$executorTimezone. Set executor timezone to $driverTimezone.") } GpuCoreDumpHandler.executorInit(conf, pluginContext) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 66803977cdb..85fdff7494a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -1204,6 +1204,10 @@ abstract class BaseExprMeta[INPUT <: Expression]( printAst(appender, 0, all) appender.toString() } + + def requireAstForGpu(): Boolean = { + false + } } abstract class ExprMeta[INPUT <: Expression]( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index 517b15d9912..1a18b166147 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -307,7 +307,7 @@ final class TypeSig private( typeMeta.dataType.foreach { dt => val expr = exprMeta.wrapped.asInstanceOf[Expression] - if (!isSupportedByPlugin(dt)) { + if (!isSupportedByPlugin(dt, !TypeChecks.isNonUtcTimeZoneEnable())) { willNotWork(s"$name expression ${expr.getClass.getSimpleName} $expr " + reasonNotSupported(dt).mkString("(", ", ", ")")) } else if (isLitOnly(dt) && !GpuOverrides.isLit(expr)) { @@ -824,6 +824,7 @@ abstract class TypeChecks[RET] { } object TypeChecks { + /** * Check if the time zone passed is supported by plugin. */ @@ -1010,10 +1011,12 @@ class ExecChecks private( // expression.toString to capture ids in not-on-GPU tags def toStructField(a: Attribute) = StructField(name = a.toString(), dataType = a.dataType) + val checkUtcTz = !TypeChecks.isNonUtcTimeZoneEnable() + tagUnsupportedTypes(meta, check, meta.outputAttributes.map(toStructField), - "unsupported data types in output: %s") + "unsupported data types in output: %s", checkUtcTz) tagUnsupportedTypes(meta, check, meta.childPlans.flatMap(_.outputAttributes.map(toStructField)), - "unsupported data types in input: %s") + "unsupported data types in input: %s", checkUtcTz) val namedChildExprs = meta.namedChildExprs diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala index 702928225a7..829a07b0b7d 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala @@ -16,11 +16,15 @@ package com.nvidia.spark.rapids +import java.sql.{Date, Timestamp} +import java.time.{ZoneId, ZonedDateTime} +import java.util.TimeZone + +import scala.collection.mutable.ListBuffer + import ai.rapids.cudf.{ColumnVector, RegexProgram} import com.nvidia.spark.rapids.Arm.withResource -import java.sql.{Date, Timestamp} import org.scalatest.BeforeAndAfterEach -import scala.collection.mutable.ListBuffer import org.apache.spark.SparkConf import org.apache.spark.sql.{Row, SparkSession} @@ -280,6 +284,48 @@ class ParseDateTimeSuite extends SparkQueryCompareTestSuite with BeforeAndAfterE assert(res) } + test("literals: ensure time literals are correct with different timezones") { + testTimeWithDiffTimezones("Asia/Shanghai", "America/New_York") + testTimeWithDiffTimezones("Asia/Shanghai", "UTC") + testTimeWithDiffTimezones("UTC", "Asia/Shanghai") + } + + private[this] def testTimeWithDiffTimezones(sessionTZStr: String, systemTZStr: String) = { + withTimeZones(sessionTimeZone = sessionTZStr, systemTimeZone = systemTZStr) { conf => + val df = withGpuSparkSession(spark => { + spark.sql("SELECT current_date(), current_timestamp(), now() FROM RANGE(1, 10)") + }, conf) + + val times = df.collect() + val zonedDateTime = ZonedDateTime.now(ZoneId.of("America/New_York")) + val res = times.forall(time => { + val diffDate = zonedDateTime.toLocalDate.toEpochDay - time.getLocalDate(0).toEpochDay + val diffTimestamp = + zonedDateTime.toInstant.getNano - time.getInstant(1).getNano + val diffNow = + zonedDateTime.toInstant.getNano - time.getInstant(2).getNano + // For date, at most 1 day difference when execution is crossing two days + // For timestamp or now, it should be less than 1 second allowing Spark's execution + diffDate.abs <= 1 & diffTimestamp.abs <= 1E9 & diffNow.abs <= 1E9 + }) + assert(res) + } + } + + private def withTimeZones(sessionTimeZone: String, + systemTimeZone: String)(f: SparkConf => Unit): Unit = { + val conf = new SparkConf() + conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sessionTimeZone) + conf.set(SQLConf.DATETIME_JAVA8API_ENABLED.key, "true") + val originTimeZone = TimeZone.getDefault + try { + TimeZone.setDefault(TimeZone.getTimeZone(systemTimeZone)) + f(conf) + } finally { + TimeZone.setDefault(originTimeZone) + } + } + private def testRegex(rule: RegexReplace, values: Seq[String], expected: Seq[String]): Unit = { withResource(ColumnVector.fromStrings(values: _*)) { v => withResource(ColumnVector.fromStrings(expected: _*)) { expected => From 36ddb24173f93378aeb618ef2fed99d942e9da20 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Fri, 17 Nov 2023 17:45:17 +0800 Subject: [PATCH 02/46] Remove type based timezone checker --- .../com/nvidia/spark/rapids/GpuCSVScan.scala | 1 - .../com/nvidia/spark/rapids/GpuCast.scala | 3 - .../spark/rapids/GpuParquetFileFormat.scala | 2 +- .../com/nvidia/spark/rapids/RapidsMeta.scala | 44 +----------- .../com/nvidia/spark/rapids/TypeChecks.scala | 72 +++---------------- .../catalyst/json/rapids/GpuJsonScan.scala | 1 - .../spark/rapids/ParseDateTimeSuite.scala | 2 +- 7 files changed, 13 insertions(+), 112 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala index 7f078cb4db2..29fb46415c2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala @@ -169,7 +169,6 @@ object GpuCSVScan { } if (types.contains(TimestampType)) { - meta.checkTimeZoneId(parsedOptions.zoneId) GpuTextBasedDateUtils.tagCudfFormat(meta, GpuCsvUtils.timestampFormatInRead(parsedOptions), parseString = true) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index 2f59cfba072..8f7ddc6f01f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -177,9 +177,6 @@ abstract class CastExprMetaBase[INPUT <: UnaryExpression with TimeZoneAwareExpre def buildTagMessage(entry: ConfEntry[_]): String = { s"${entry.doc}. To enable this operation on the GPU, set ${entry.key} to true." } - - // timezone tagging in type checks is good enough, so always false - override protected val needTimezoneTagging: Boolean = false } object CastOptions { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala index 7e845491ec0..22f0fef2262 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala @@ -129,7 +129,7 @@ object GpuParquetFileFormat { SparkShimImpl.parquetRebaseWrite(sqlConf)) if ((int96RebaseMode == DateTimeRebaseLegacy || dateTimeRebaseMode == DateTimeRebaseLegacy) - && !TypeChecks.areTimestampsSupported()) { + && !TypeChecks.isUTCTimezone()) { meta.willNotWorkOnGpu("Only UTC timezone is supported in LEGACY rebase mode. " + s"Current timezone settings: (JVM : ${ZoneId.systemDefault()}, " + s"session: ${SQLConf.get.sessionLocalTimeZone}). " + diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 85fdff7494a..47f1306f61b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -16,13 +16,11 @@ package com.nvidia.spark.rapids -import java.time.ZoneId - import scala.collection.mutable import com.nvidia.spark.rapids.shims.{DistributionUtil, SparkShimImpl} -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BinaryExpression, ComplexTypeMergingExpression, Expression, QuaternaryExpression, String2TrimExpression, TernaryExpression, TimeZoneAwareExpression, UnaryExpression, WindowExpression, WindowFunction} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BinaryExpression, ComplexTypeMergingExpression, Expression, QuaternaryExpression, String2TrimExpression, TernaryExpression, UnaryExpression, WindowExpression, WindowFunction} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, ImperativeAggregate, TypedImperativeAggregate} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeTag @@ -380,19 +378,6 @@ abstract class RapidsMeta[INPUT <: BASE, BASE, OUTPUT <: BASE]( } } - def checkTimeZoneId(sessionZoneId: ZoneId): Unit = { - // Both of the Spark session time zone and JVM's default time zone should be UTC. - if (!TypeChecks.areTimestampsSupported(sessionZoneId)) { - willNotWorkOnGpu("Only UTC zone id is supported. " + - s"Actual session local zone id: $sessionZoneId") - } - - val defaultZoneId = ZoneId.systemDefault() - if (!TypeChecks.areTimestampsSupported(defaultZoneId)) { - willNotWorkOnGpu(s"Only UTC zone id is supported. Actual default zone id: $defaultZoneId") - } - } - /** * Create a string representation of this in append. * @param strBuilder where to place the string representation. @@ -1082,22 +1067,8 @@ abstract class BaseExprMeta[INPUT <: Expression]( val isFoldableNonLitAllowed: Boolean = false - /** - * Whether to tag a TimeZoneAwareExpression for timezone after all the other tagging - * is done. - * By default a TimeZoneAwareExpression always requires the timezone tagging, but - * there are some exceptions, e.g. 'Cast', who requires timezone tagging only when it - * has timezone sensitive type as input or output. - * - * Override this to match special cases. - */ - protected def needTimezoneTagging: Boolean = { - // A TimeZoneAwareExpression with no timezone sensitive types as input/output will - // escape from the timezone tagging in the prior type checks. So ask for tagging here. - // e.g. 'UnixTimestamp' with 'DateType' as the input, timezone will be taken into - // account when converting a Date to a Long. - !(dataType +: childExprs.map(_.dataType)).exists(TypeChecks.isTimezoneSensitiveType) - } + // Default false as conservative approach to allow timezone related expression converted to GPU + lazy val isTimezoneSupported: Boolean = false final override def tagSelfForGpu(): Unit = { if (wrapped.foldable && !GpuOverrides.isLit(wrapped) && !isFoldableNonLitAllowed) { @@ -1106,11 +1077,6 @@ abstract class BaseExprMeta[INPUT <: Expression]( } rule.getChecks.foreach(_.tag(this)) tagExprForGpu() - wrapped match { - case tzAware: TimeZoneAwareExpression if needTimezoneTagging => - checkTimeZoneId(tzAware.zoneId) - case _ => // do nothing - } } /** @@ -1204,10 +1170,6 @@ abstract class BaseExprMeta[INPUT <: Expression]( printAst(appender, 0, all) appender.toString() } - - def requireAstForGpu(): Boolean = { - false - } } abstract class ExprMeta[INPUT <: Expression]( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index 1a18b166147..53283d4be1f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -17,7 +17,6 @@ package com.nvidia.spark.rapids import java.io.{File, FileOutputStream} -import java.time.ZoneId import ai.rapids.cudf.DType import com.nvidia.spark.rapids.shims.{CastCheckShims, GpuTypeShims, TypeSigUtil} @@ -307,7 +306,7 @@ final class TypeSig private( typeMeta.dataType.foreach { dt => val expr = exprMeta.wrapped.asInstanceOf[Expression] - if (!isSupportedByPlugin(dt, !TypeChecks.isNonUtcTimeZoneEnable())) { + if (!isSupportedByPlugin(dt)) { willNotWork(s"$name expression ${expr.getClass.getSimpleName} $expr " + reasonNotSupported(dt).mkString("(", ", ", ")")) } else if (isLitOnly(dt) && !GpuOverrides.isLit(expr)) { @@ -364,7 +363,7 @@ final class TypeSig private( case DoubleType => check.contains(TypeEnum.DOUBLE) case DateType => check.contains(TypeEnum.DATE) case TimestampType if check.contains(TypeEnum.TIMESTAMP) => - TypeChecks.areTimestampsSupported() + check.contains(TypeEnum.TIMESTAMP) case StringType => check.contains(TypeEnum.STRING) case dt: DecimalType => check.contains(TypeEnum.DECIMAL) && @@ -402,15 +401,6 @@ final class TypeSig private( } } - private[this] def timezoneNotSupportedMessage(dataType: DataType, - te: TypeEnum.Value, check: TypeEnum.ValueSet, isChild: Boolean): Seq[String] = { - if (check.contains(te) && !TypeChecks.areTimestampsSupported()) { - Seq(withChild(isChild, TypeChecks.timezoneNotSupportedString(dataType))) - } else { - basicNotSupportedMessage(dataType, te, check, isChild) - } - } - private[this] def reasonNotSupported( check: TypeEnum.ValueSet, dataType: DataType, @@ -433,7 +423,7 @@ final class TypeSig private( case DateType => basicNotSupportedMessage(dataType, TypeEnum.DATE, check, isChild) case TimestampType => - timezoneNotSupportedMessage(dataType, TypeEnum.TIMESTAMP, check, isChild) + basicNotSupportedMessage(dataType, TypeEnum.TIMESTAMP, check, isChild) case StringType => basicNotSupportedMessage(dataType, TypeEnum.STRING, check, isChild) case dt: DecimalType => @@ -780,30 +770,6 @@ abstract class TypeChecks[RET] { }.mkString(", ") } - /** - * Original log does not print enough info when timezone is not UTC, - * here check again to add UTC info. - */ - private def tagTimezoneInfoIfHasTimestampType( - unsupportedTypes: Map[DataType, Set[String]], - meta: RapidsMeta[_, _, _]): Unit = { - def checkTimestampType(dataType: DataType): Unit = dataType match { - case TimestampType if !TypeChecks.areTimestampsSupported() => - meta.willNotWorkOnGpu(TypeChecks.timezoneNotSupportedString(dataType)) - case ArrayType(elementType, _) => - checkTimestampType(elementType) - case MapType(keyType, valueType, _) => - checkTimestampType(keyType) - checkTimestampType(valueType) - case StructType(fields) => - fields.foreach(field => checkTimestampType(field.dataType)) - case _ => // do nothing - } - unsupportedTypes.foreach { case (dataType, _) => - checkTimestampType(dataType) - } - } - protected def tagUnsupportedTypes( meta: RapidsMeta[_, _, _], sig: TypeSig, @@ -815,8 +781,6 @@ abstract class TypeChecks[RET] { .groupBy(_.dataType) .mapValues(_.map(_.name).toSet).toMap - tagTimezoneInfoIfHasTimestampType(unsupportedTypes, meta) - if (unsupportedTypes.nonEmpty) { meta.willNotWorkOnGpu(msgFormat.format(stringifyTypeAttributeMap(unsupportedTypes))) } @@ -825,32 +789,14 @@ abstract class TypeChecks[RET] { object TypeChecks { - /** - * Check if the time zone passed is supported by plugin. - */ - def areTimestampsSupported(timezoneId: ZoneId): Boolean = { - timezoneId.normalized() == GpuOverrides.UTC_TIMEZONE_ID - } - - def areTimestampsSupported(zoneIdString: String): Boolean = { - val zoneId = DateTimeUtils.getZoneId(zoneIdString) - areTimestampsSupported(zoneId) - } - - def areTimestampsSupported(): Boolean = { - areTimestampsSupported(ZoneId.systemDefault()) && - areTimestampsSupported(SQLConf.get.sessionLocalTimeZone) + def isUTCTimezone(): Boolean = { + val zoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone) + zoneId.normalized() == GpuOverrides.UTC_TIMEZONE_ID } def isTimezoneSensitiveType(dataType: DataType): Boolean = { dataType == TimestampType } - - def timezoneNotSupportedString(dataType: DataType): String = { - s"$dataType is not supported with timezone settings: (JVM:" + - s" ${ZoneId.systemDefault()}, session: ${SQLConf.get.sessionLocalTimeZone})." + - s" Set both of the timezones to UTC to enable $dataType support" - } } /** @@ -1011,12 +957,10 @@ class ExecChecks private( // expression.toString to capture ids in not-on-GPU tags def toStructField(a: Attribute) = StructField(name = a.toString(), dataType = a.dataType) - val checkUtcTz = !TypeChecks.isNonUtcTimeZoneEnable() - tagUnsupportedTypes(meta, check, meta.outputAttributes.map(toStructField), - "unsupported data types in output: %s", checkUtcTz) + "unsupported data types in output: %s") tagUnsupportedTypes(meta, check, meta.childPlans.flatMap(_.outputAttributes.map(toStructField)), - "unsupported data types in input: %s", checkUtcTz) + "unsupported data types in input: %s") val namedChildExprs = meta.namedChildExprs diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala index c4840839616..7b288037725 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala @@ -169,7 +169,6 @@ object GpuJsonScan { } if (types.contains(TimestampType)) { - meta.checkTimeZoneId(parsedOptions.zoneId) GpuTextBasedDateUtils.tagCudfFormat(meta, GpuJsonUtils.timestampFormatInRead(parsedOptions), parseString = true) } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala index 829a07b0b7d..74ef6e5cc73 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala @@ -17,7 +17,7 @@ package com.nvidia.spark.rapids import java.sql.{Date, Timestamp} -import java.time.{ZoneId, ZonedDateTime} +import java.time.{ZonedDateTime, ZoneId} import java.util.TimeZone import scala.collection.mutable.ListBuffer From 28ba19ed944ff2e3eeb1d104448608f9fe3c7b7e Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Fri, 17 Nov 2023 21:06:31 +0800 Subject: [PATCH 03/46] Add timezone checker for expressions --- .../com/nvidia/spark/rapids/GpuCSVScan.scala | 1 + .../com/nvidia/spark/rapids/RapidsMeta.scala | 55 ++++++++++++++++++- .../com/nvidia/spark/rapids/TypeChecks.scala | 12 +++- .../catalyst/json/rapids/GpuJsonScan.scala | 1 + 4 files changed, 64 insertions(+), 5 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala index 29fb46415c2..7f078cb4db2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala @@ -169,6 +169,7 @@ object GpuCSVScan { } if (types.contains(TimestampType)) { + meta.checkTimeZoneId(parsedOptions.zoneId) GpuTextBasedDateUtils.tagCudfFormat(meta, GpuCsvUtils.timestampFormatInRead(parsedOptions), parseString = true) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 47f1306f61b..976898c6758 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -16,11 +16,13 @@ package com.nvidia.spark.rapids +import java.time.ZoneId + import scala.collection.mutable import com.nvidia.spark.rapids.shims.{DistributionUtil, SparkShimImpl} -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BinaryExpression, ComplexTypeMergingExpression, Expression, QuaternaryExpression, String2TrimExpression, TernaryExpression, UnaryExpression, WindowExpression, WindowFunction} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BinaryExpression, ComplexTypeMergingExpression, Expression, QuaternaryExpression, String2TrimExpression, TernaryExpression, TimeZoneAwareExpression, UnaryExpression, UTCTimestamp, WindowExpression, WindowFunction} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, ImperativeAggregate, TypedImperativeAggregate} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeTag @@ -378,6 +380,19 @@ abstract class RapidsMeta[INPUT <: BASE, BASE, OUTPUT <: BASE]( } } + def checkTimeZoneId(sessionZoneId: ZoneId): Unit = { + // Both of the Spark session time zone and JVM's default time zone should be UTC. + if (!TypeChecks.isTimestampsSupported(sessionZoneId)) { + willNotWorkOnGpu("Only UTC zone id is supported. " + + s"Actual session local zone id: $sessionZoneId") + } + + val defaultZoneId = ZoneId.systemDefault() + if (!TypeChecks.isTimestampsSupported(defaultZoneId)) { + willNotWorkOnGpu(s"Only UTC zone id is supported. Actual default zone id: $defaultZoneId") + } + } + /** * Create a string representation of this in append. * @param strBuilder where to place the string representation. @@ -1067,8 +1082,23 @@ abstract class BaseExprMeta[INPUT <: Expression]( val isFoldableNonLitAllowed: Boolean = false - // Default false as conservative approach to allow timezone related expression converted to GPU - lazy val isTimezoneSupported: Boolean = false + // Whether timezone is supported for those expressions needs to be check. + // TODO: use TimezoneDB Utils to tell whether timezone is supported + val isTimezoneSupported: Boolean = false + + //+------------------------+-------------------+-----------------------------------------+ + //| Value | needTimezoneCheck | isTimezoneSupported | + //+------------------------+-------------------+-----------------------------------------+ + //| TimezoneAwareExpression| True | False by default, True when implemented | + //| UTCTimestamp | True | False by default, True when implemented | + //| Others | False | N/A (will not be checked) | + //+------------------------+-------------------+-----------------------------------------+ + lazy val needTimezoneCheck: Boolean = { + wrapped match { + case _: TimeZoneAwareExpression | _: UTCTimestamp => true + case _ => false + } + } final override def tagSelfForGpu(): Unit = { if (wrapped.foldable && !GpuOverrides.isLit(wrapped) && !isFoldableNonLitAllowed) { @@ -1076,9 +1106,28 @@ abstract class BaseExprMeta[INPUT <: Expression]( s"$wrapped is foldable and operates on non literals") } rule.getChecks.foreach(_.tag(this)) + if (needTimezoneCheck && !isTimezoneSupported) checkTimestampType(dataType, this) tagExprForGpu() } + /** + * Check whether contains timestamp type and whether timezone is supported + */ + def checkTimestampType(dataType: DataType, meta: RapidsMeta[_, _, _]): Unit = { + dataType match { + case TimestampType if !TypeChecks.isUTCTimezone() => + meta.willNotWorkOnGpu(TypeChecks.timezoneNotSupportedString(dataType)) + case ArrayType(elementType, _) => + checkTimestampType(elementType, meta) + case MapType(keyType, valueType, _) => + checkTimestampType(keyType, meta) + checkTimestampType(valueType, meta) + case StructType(fields) => + fields.foreach(field => checkTimestampType(field.dataType, meta)) + case _ => // do nothing + } + } + /** * Called to verify that this expression will work on the GPU. For most expressions without * extra checks all of the checks should have already been done. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index 53283d4be1f..530b5a71177 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import java.io.{File, FileOutputStream} +import java.time.ZoneId import ai.rapids.cudf.DType import com.nvidia.spark.rapids.shims.{CastCheckShims, GpuTypeShims, TypeSigUtil} @@ -789,13 +790,20 @@ abstract class TypeChecks[RET] { object TypeChecks { + // TODO: move this to Timezone DB + def isTimestampsSupported(timezoneId: ZoneId): Boolean = { + timezoneId.normalized() == GpuOverrides.UTC_TIMEZONE_ID + } + def isUTCTimezone(): Boolean = { val zoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone) zoneId.normalized() == GpuOverrides.UTC_TIMEZONE_ID } - def isTimezoneSensitiveType(dataType: DataType): Boolean = { - dataType == TimestampType + def timezoneNotSupportedString(dataType: DataType): String = { + s"$dataType is not supported with timezone settings: (JVM:" + + s" ${ZoneId.systemDefault()}, session: ${SQLConf.get.sessionLocalTimeZone})." + + s" Set both of the timezones to UTC to enable $dataType support" } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala index 7b288037725..c4840839616 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala @@ -169,6 +169,7 @@ object GpuJsonScan { } if (types.contains(TimestampType)) { + meta.checkTimeZoneId(parsedOptions.zoneId) GpuTextBasedDateUtils.tagCudfFormat(meta, GpuJsonUtils.timestampFormatInRead(parsedOptions), parseString = true) } From 3d5d29775c747edb8d16494fca61bb47bbe6ed85 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Fri, 17 Nov 2023 22:04:30 +0800 Subject: [PATCH 04/46] Add check for cast --- .../scala/com/nvidia/spark/rapids/RapidsMeta.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 976898c6758..921133c6065 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -17,12 +17,11 @@ package com.nvidia.spark.rapids import java.time.ZoneId - import scala.collection.mutable import com.nvidia.spark.rapids.shims.{DistributionUtil, SparkShimImpl} -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BinaryExpression, ComplexTypeMergingExpression, Expression, QuaternaryExpression, String2TrimExpression, TernaryExpression, TimeZoneAwareExpression, UnaryExpression, UTCTimestamp, WindowExpression, WindowFunction} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BinaryExpression, Cast, ComplexTypeMergingExpression, Expression, QuaternaryExpression, String2TrimExpression, TernaryExpression, TimeZoneAwareExpression, UTCTimestamp, UnaryExpression, WindowExpression, WindowFunction} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, ImperativeAggregate, TypedImperativeAggregate} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeTag @@ -1086,6 +1085,9 @@ abstract class BaseExprMeta[INPUT <: Expression]( // TODO: use TimezoneDB Utils to tell whether timezone is supported val isTimezoneSupported: Boolean = false + // Both [[isTimezoneSupported]] and [[needTimezoneCheck]] are needed to check whether timezone + // check needed. For cast expression, only some cases are needed pending on its data type and + // its child's data type. //+------------------------+-------------------+-----------------------------------------+ //| Value | needTimezoneCheck | isTimezoneSupported | //+------------------------+-------------------+-----------------------------------------+ @@ -1095,7 +1097,9 @@ abstract class BaseExprMeta[INPUT <: Expression]( //+------------------------+-------------------+-----------------------------------------+ lazy val needTimezoneCheck: Boolean = { wrapped match { - case _: TimeZoneAwareExpression | _: UTCTimestamp => true + case _: TimeZoneAwareExpression => + if (wrapped.isInstanceOf[Cast]) wrapped.asInstanceOf[Cast].needsTimeZone else true + case _: UTCTimestamp => true case _ => false } } From b3f85bd16f099963800629e8c87faacc249c5299 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Fri, 17 Nov 2023 22:11:42 +0800 Subject: [PATCH 05/46] code style fix --- .../src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 921133c6065..3e78c76700c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -17,11 +17,12 @@ package com.nvidia.spark.rapids import java.time.ZoneId + import scala.collection.mutable import com.nvidia.spark.rapids.shims.{DistributionUtil, SparkShimImpl} -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BinaryExpression, Cast, ComplexTypeMergingExpression, Expression, QuaternaryExpression, String2TrimExpression, TernaryExpression, TimeZoneAwareExpression, UTCTimestamp, UnaryExpression, WindowExpression, WindowFunction} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BinaryExpression, Cast, ComplexTypeMergingExpression, Expression, QuaternaryExpression, String2TrimExpression, TernaryExpression, TimeZoneAwareExpression, UnaryExpression, UTCTimestamp, WindowExpression, WindowFunction} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, ImperativeAggregate, TypedImperativeAggregate} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeTag From c7c60d9117b0374a090141b1c769fe4c443517ca Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Mon, 20 Nov 2023 12:11:17 +0800 Subject: [PATCH 06/46] Fix premerge fails --- .../com/nvidia/spark/rapids/RapidsMeta.scala | 18 +++++------------- .../com/nvidia/spark/rapids/TypeChecks.scala | 6 +++--- 2 files changed, 8 insertions(+), 16 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 3e78c76700c..de1b03a1991 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -1111,25 +1111,17 @@ abstract class BaseExprMeta[INPUT <: Expression]( s"$wrapped is foldable and operates on non literals") } rule.getChecks.foreach(_.tag(this)) - if (needTimezoneCheck && !isTimezoneSupported) checkTimestampType(dataType, this) + if (needTimezoneCheck && !isTimezoneSupported) checkTimestampType(this) tagExprForGpu() } /** * Check whether contains timestamp type and whether timezone is supported */ - def checkTimestampType(dataType: DataType, meta: RapidsMeta[_, _, _]): Unit = { - dataType match { - case TimestampType if !TypeChecks.isUTCTimezone() => - meta.willNotWorkOnGpu(TypeChecks.timezoneNotSupportedString(dataType)) - case ArrayType(elementType, _) => - checkTimestampType(elementType, meta) - case MapType(keyType, valueType, _) => - checkTimestampType(keyType, meta) - checkTimestampType(valueType, meta) - case StructType(fields) => - fields.foreach(field => checkTimestampType(field.dataType, meta)) - case _ => // do nothing + def checkTimestampType(meta: RapidsMeta[_, _, _]): Unit = { + // FIXME: use new API from TimezoneDB utils to check whether it's supported + if (!TypeChecks.isUTCTimezone()) { + meta.willNotWorkOnGpu(TypeChecks.timezoneNotSupportedString(meta.wrapped.getClass.toString)) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index 530b5a71177..63a63ad3574 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -800,10 +800,10 @@ object TypeChecks { zoneId.normalized() == GpuOverrides.UTC_TIMEZONE_ID } - def timezoneNotSupportedString(dataType: DataType): String = { - s"$dataType is not supported with timezone settings: (JVM:" + + def timezoneNotSupportedString(exprName: String): String = { + s"$exprName is not supported with timezone settings: (JVM:" + s" ${ZoneId.systemDefault()}, session: ${SQLConf.get.sessionLocalTimeZone})." + - s" Set both of the timezones to UTC to enable $dataType support" + s" Set both of the timezones to UTC to enable $exprName support" } } From 90d975e31a5940f98b3556df76e313c7d447c745 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Mon, 20 Nov 2023 16:36:08 +0800 Subject: [PATCH 07/46] Fix --- .../src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index 63a63ad3574..227b7158046 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -363,8 +363,7 @@ final class TypeSig private( case FloatType => check.contains(TypeEnum.FLOAT) case DoubleType => check.contains(TypeEnum.DOUBLE) case DateType => check.contains(TypeEnum.DATE) - case TimestampType if check.contains(TypeEnum.TIMESTAMP) => - check.contains(TypeEnum.TIMESTAMP) + case TimestampType => check.contains(TypeEnum.TIMESTAMP) case StringType => check.contains(TypeEnum.STRING) case dt: DecimalType => check.contains(TypeEnum.DECIMAL) && From b88a13b16e36c35a3fc8361af36ccd037b57946a Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Tue, 21 Nov 2023 15:01:42 +0800 Subject: [PATCH 08/46] Refine comments --- .../src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala | 3 --- .../src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala | 1 + 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index de1b03a1991..abbea721826 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -1115,9 +1115,6 @@ abstract class BaseExprMeta[INPUT <: Expression]( tagExprForGpu() } - /** - * Check whether contains timestamp type and whether timezone is supported - */ def checkTimestampType(meta: RapidsMeta[_, _, _]): Unit = { // FIXME: use new API from TimezoneDB utils to check whether it's supported if (!TypeChecks.isUTCTimezone()) { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index 227b7158046..d9d891cb7f0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -799,6 +799,7 @@ object TypeChecks { zoneId.normalized() == GpuOverrides.UTC_TIMEZONE_ID } + // TODO: change the string about supported timezones def timezoneNotSupportedString(exprName: String): String = { s"$exprName is not supported with timezone settings: (JVM:" + s" ${ZoneId.systemDefault()}, session: ${SQLConf.get.sessionLocalTimeZone})." + From 3cda255cd223db636e1e0f8ec8e63950754d8dfb Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Tue, 21 Nov 2023 21:00:58 +0800 Subject: [PATCH 09/46] Refine comments --- .../scala/com/nvidia/spark/rapids/RapidsMeta.scala | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index abbea721826..eea62ca99cb 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -1082,13 +1082,15 @@ abstract class BaseExprMeta[INPUT <: Expression]( val isFoldableNonLitAllowed: Boolean = false - // Whether timezone is supported for those expressions needs to be check. - // TODO: use TimezoneDB Utils to tell whether timezone is supported + // This is a toggle flag whether to fallback to previous UTC-only check. When one expression + // is supported, it needs to override this flag to bypass UTC check. However, timezone check is + // still needed within `tagExprForGpu` method if desired timezone is supported by Gpu kernel. val isTimezoneSupported: Boolean = false // Both [[isTimezoneSupported]] and [[needTimezoneCheck]] are needed to check whether timezone // check needed. For cast expression, only some cases are needed pending on its data type and // its child's data type. + // //+------------------------+-------------------+-----------------------------------------+ //| Value | needTimezoneCheck | isTimezoneSupported | //+------------------------+-------------------+-----------------------------------------+ @@ -1115,8 +1117,11 @@ abstract class BaseExprMeta[INPUT <: Expression]( tagExprForGpu() } + /** + * Timezone check which only allows UTC timezone. This is consistent with previous behavior. + * @param meta to check whether it's UTC + */ def checkTimestampType(meta: RapidsMeta[_, _, _]): Unit = { - // FIXME: use new API from TimezoneDB utils to check whether it's supported if (!TypeChecks.isUTCTimezone()) { meta.willNotWorkOnGpu(TypeChecks.timezoneNotSupportedString(meta.wrapped.getClass.toString)) } From 42c78887e7cb3f1f78c1d72884391ce40c739241 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Tue, 21 Nov 2023 21:27:26 +0800 Subject: [PATCH 10/46] Refactor --- .../spark/rapids/GpuParquetFileFormat.scala | 3 ++- .../com/nvidia/spark/rapids/RapidsMeta.scala | 9 ++++---- .../com/nvidia/spark/rapids/TypeChecks.scala | 23 ------------------- .../apache/spark/sql/rapids/TimeZoneDB.scala | 17 ++++++++++++++ 4 files changed, 24 insertions(+), 28 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala index 22f0fef2262..c694a0a3478 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.execution.datasources.DataSourceUtils import org.apache.spark.sql.execution.datasources.parquet.{ParquetOptions, ParquetWriteSupport} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType +import org.apache.spark.sql.rapids.TimeZoneDB import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch @@ -129,7 +130,7 @@ object GpuParquetFileFormat { SparkShimImpl.parquetRebaseWrite(sqlConf)) if ((int96RebaseMode == DateTimeRebaseLegacy || dateTimeRebaseMode == DateTimeRebaseLegacy) - && !TypeChecks.isUTCTimezone()) { + && !TimeZoneDB.isUTCTimezone()) { meta.willNotWorkOnGpu("Only UTC timezone is supported in LEGACY rebase mode. " + s"Current timezone settings: (JVM : ${ZoneId.systemDefault()}, " + s"session: ${SQLConf.get.sessionLocalTimeZone}). " + diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index eea62ca99cb..4ff1402e1de 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.execution.command.{DataWritingCommand, RunnableComma import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} import org.apache.spark.sql.execution.python.AggregateInPandasExec +import org.apache.spark.sql.rapids.TimeZoneDB import org.apache.spark.sql.rapids.aggregate.{CpuToGpuAggregateBufferConverter, GpuToCpuAggregateBufferConverter} import org.apache.spark.sql.rapids.execution.{GpuBroadcastHashJoinMetaBase, GpuBroadcastNestedLoopJoinMetaBase} import org.apache.spark.sql.types.DataType @@ -382,13 +383,13 @@ abstract class RapidsMeta[INPUT <: BASE, BASE, OUTPUT <: BASE]( def checkTimeZoneId(sessionZoneId: ZoneId): Unit = { // Both of the Spark session time zone and JVM's default time zone should be UTC. - if (!TypeChecks.isTimestampsSupported(sessionZoneId)) { + if (!TimeZoneDB.isSupportedTimezone(sessionZoneId)) { willNotWorkOnGpu("Only UTC zone id is supported. " + s"Actual session local zone id: $sessionZoneId") } val defaultZoneId = ZoneId.systemDefault() - if (!TypeChecks.isTimestampsSupported(defaultZoneId)) { + if (!TimeZoneDB.isSupportedTimezone(defaultZoneId)) { willNotWorkOnGpu(s"Only UTC zone id is supported. Actual default zone id: $defaultZoneId") } } @@ -1122,8 +1123,8 @@ abstract class BaseExprMeta[INPUT <: Expression]( * @param meta to check whether it's UTC */ def checkTimestampType(meta: RapidsMeta[_, _, _]): Unit = { - if (!TypeChecks.isUTCTimezone()) { - meta.willNotWorkOnGpu(TypeChecks.timezoneNotSupportedString(meta.wrapped.getClass.toString)) + if (!TimeZoneDB.isUTCTimezone()) { + meta.willNotWorkOnGpu(TimeZoneDB.timezoneNotSupportedString(meta.wrapped.getClass.toString)) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index d9d891cb7f0..b8dea05b73f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -17,14 +17,11 @@ package com.nvidia.spark.rapids import java.io.{File, FileOutputStream} -import java.time.ZoneId import ai.rapids.cudf.DType import com.nvidia.spark.rapids.shims.{CastCheckShims, GpuTypeShims, TypeSigUtil} import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, UnaryExpression, WindowSpecDefinition} -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** Trait of TypeSigUtil for different spark versions */ @@ -787,26 +784,6 @@ abstract class TypeChecks[RET] { } } -object TypeChecks { - - // TODO: move this to Timezone DB - def isTimestampsSupported(timezoneId: ZoneId): Boolean = { - timezoneId.normalized() == GpuOverrides.UTC_TIMEZONE_ID - } - - def isUTCTimezone(): Boolean = { - val zoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone) - zoneId.normalized() == GpuOverrides.UTC_TIMEZONE_ID - } - - // TODO: change the string about supported timezones - def timezoneNotSupportedString(exprName: String): String = { - s"$exprName is not supported with timezone settings: (JVM:" + - s" ${ZoneId.systemDefault()}, session: ${SQLConf.get.sessionLocalTimeZone})." + - s" Set both of the timezones to UTC to enable $exprName support" - } -} - /** * Checks a set of named inputs to an SparkPlan node against a TypeSig */ diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/TimeZoneDB.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/TimeZoneDB.scala index 2b1b8e6576b..cc2a011f92d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/TimeZoneDB.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/TimeZoneDB.scala @@ -23,12 +23,18 @@ import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.GpuOverrides import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.internal.SQLConf object TimeZoneDB { def isUTCTimezone(timezoneId: ZoneId): Boolean = { timezoneId.normalized() == GpuOverrides.UTC_TIMEZONE_ID } + def isUTCTimezone(): Boolean = { + val zoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone) + isUTCTimezone(zoneId.normalized()) + } + // Copied from Spark. Used to format time zone ID string with (+|-)h:mm and (+|-)hh:m def getZoneId(timezoneId: String): ZoneId = { val formattedZoneId = timezoneId @@ -45,6 +51,17 @@ object TimeZoneDB { rules.isFixedOffset || rules.getTransitionRules.isEmpty } + def isSupportedTimezone(timezoneId: ZoneId): Boolean = { + val rules = timezoneId.getRules + rules.isFixedOffset || rules.getTransitionRules.isEmpty + } + + def timezoneNotSupportedString(exprName: String): String = { + s"$exprName is not supported with timezone settings: (JVM:" + + s" ${ZoneId.systemDefault()}, session: ${SQLConf.get.sessionLocalTimeZone})." + + s" Set both of the timezones to UTC to enable $exprName support" + } + def cacheDatabase(): Unit = {} /** From 1da12912419b5e2e585109475cbed03c362dd9b2 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 22 Nov 2023 14:09:00 +0800 Subject: [PATCH 11/46] Refine comments --- .../com/nvidia/spark/rapids/RapidsMeta.scala | 72 +++++++++++++++---- .../apache/spark/sql/rapids/TimeZoneDB.scala | 7 +- 2 files changed, 63 insertions(+), 16 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 4ff1402e1de..8f70e7c9cdc 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -1083,11 +1083,45 @@ abstract class BaseExprMeta[INPUT <: Expression]( val isFoldableNonLitAllowed: Boolean = false - // This is a toggle flag whether to fallback to previous UTC-only check. When one expression - // is supported, it needs to override this flag to bypass UTC check. However, timezone check is - // still needed within `tagExprForGpu` method if desired timezone is supported by Gpu kernel. - val isTimezoneSupported: Boolean = false + // There are 4 levels of timezone check in GPU plan tag phase: + // Level 1: Check whether an expression is related to timezone. This is achieved by + // [[needTimezoneCheck]] below. + // Level 2: Check on golden configuration 'spark.rapids.sql.nonUtc.enabled'. If + // yes, we pass to next level timezone check. If not, we only pass UTC case as before. + // Level 3: Check related expression has been implemented with timezone. There is a + // toggle flag [[isTimezoneSupported]] for this. If false, fallback to UTC-only check as + // before. If yes, move to next level check. When we add timezone support for a related + // function. [[isTimezoneSupported]] should be override as true. This check happens within + // Level 4: Check whether the desired timezone is supported by Gpu kernel. + def checkExprForTimezone(): Unit = { + // Level 1 check + if (!needTimezoneCheck) return + + // TODO: Level 2 check + + // Level 3 check + if (!isTimezoneSupported) return checkUTCTimezone(this) + + // Level 4 check + if (TimeZoneDB.isSupportedTimezone(getZoneId())) { + willNotWorkOnGpu(TimeZoneDB.timezoneNotSupportedStr(this.wrapped.getClass.toString)) + } + } + + protected def getZoneId(): ZoneId = { + this.wrapped match { + case tzExpr: TimeZoneAwareExpression => tzExpr.zoneId + case ts: UTCTimestamp => { + assert(false, s"Have to override getZoneId() of BaseExprMeta in ${this.getClass.toString}") + throw new IllegalArgumentException(s"Failed to get zone id from ${ts.getClass.toString}") + } + case _ => throw new IllegalArgumentException( + s"Zone check should never been happened to ${this.getClass.toString} " + + "which is not timezone related") + } + } + // Level 1 timezone checking flag // Both [[isTimezoneSupported]] and [[needTimezoneCheck]] are needed to check whether timezone // check needed. For cast expression, only some cases are needed pending on its data type and // its child's data type. @@ -1102,32 +1136,40 @@ abstract class BaseExprMeta[INPUT <: Expression]( lazy val needTimezoneCheck: Boolean = { wrapped match { case _: TimeZoneAwareExpression => + // CurrentDate expression will not go through this even it's a `TimeZoneAwareExpression`. + // It will be treated as literal in Rapids. if (wrapped.isInstanceOf[Cast]) wrapped.asInstanceOf[Cast].needsTimeZone else true case _: UTCTimestamp => true case _ => false } } - final override def tagSelfForGpu(): Unit = { - if (wrapped.foldable && !GpuOverrides.isLit(wrapped) && !isFoldableNonLitAllowed) { - willNotWorkOnGpu(s"Cannot run on GPU. Is ConstantFolding excluded? Expression " + - s"$wrapped is foldable and operates on non literals") - } - rule.getChecks.foreach(_.tag(this)) - if (needTimezoneCheck && !isTimezoneSupported) checkTimestampType(this) - tagExprForGpu() - } + // Level 3 timezone checking flag, need to override to true when supports timezone in functions + // Useless if it's not timezone related expression defined in [[needTimezoneCheck]] + val isTimezoneSupported: Boolean = false /** * Timezone check which only allows UTC timezone. This is consistent with previous behavior. + * * @param meta to check whether it's UTC */ - def checkTimestampType(meta: RapidsMeta[_, _, _]): Unit = { + def checkUTCTimezone(meta: RapidsMeta[_, _, _]): Unit = { if (!TimeZoneDB.isUTCTimezone()) { - meta.willNotWorkOnGpu(TimeZoneDB.timezoneNotSupportedString(meta.wrapped.getClass.toString)) + meta.willNotWorkOnGpu( + TimeZoneDB.nonUTCTimezoneNotSupportedStr(meta.wrapped.getClass.toString)) } } + final override def tagSelfForGpu(): Unit = { + if (wrapped.foldable && !GpuOverrides.isLit(wrapped) && !isFoldableNonLitAllowed) { + willNotWorkOnGpu(s"Cannot run on GPU. Is ConstantFolding excluded? Expression " + + s"$wrapped is foldable and operates on non literals") + } + rule.getChecks.foreach(_.tag(this)) + checkExprForTimezone() + tagExprForGpu() + } + /** * Called to verify that this expression will work on the GPU. For most expressions without * extra checks all of the checks should have already been done. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/TimeZoneDB.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/TimeZoneDB.scala index cc2a011f92d..a55ac6a54fe 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/TimeZoneDB.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/TimeZoneDB.scala @@ -56,12 +56,17 @@ object TimeZoneDB { rules.isFixedOffset || rules.getTransitionRules.isEmpty } - def timezoneNotSupportedString(exprName: String): String = { + def nonUTCTimezoneNotSupportedStr(exprName: String): String = { s"$exprName is not supported with timezone settings: (JVM:" + s" ${ZoneId.systemDefault()}, session: ${SQLConf.get.sessionLocalTimeZone})." + s" Set both of the timezones to UTC to enable $exprName support" } + def timezoneNotSupportedStr(timezoneIdStr: String): String = { + s"Timezone $timezoneIdStr is not supported yet. Only Non DST (daylight saving time) timezone" + + s" is supported." + } + def cacheDatabase(): Unit = {} /** From 46dbe60a7d193e1cceb73e00ecb65208f41c2f8c Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 22 Nov 2023 14:12:24 +0800 Subject: [PATCH 12/46] Typo --- .../src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 8f70e7c9cdc..4d2e8605c52 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -1091,7 +1091,7 @@ abstract class BaseExprMeta[INPUT <: Expression]( // Level 3: Check related expression has been implemented with timezone. There is a // toggle flag [[isTimezoneSupported]] for this. If false, fallback to UTC-only check as // before. If yes, move to next level check. When we add timezone support for a related - // function. [[isTimezoneSupported]] should be override as true. This check happens within + // function. [[isTimezoneSupported]] should be override as true. // Level 4: Check whether the desired timezone is supported by Gpu kernel. def checkExprForTimezone(): Unit = { // Level 1 check From c954125461ada98391b24cb4fd79ce8cd10ff8ea Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Mon, 27 Nov 2023 15:29:58 +0800 Subject: [PATCH 13/46] Re-enable failed test cases --- integration_tests/src/main/python/aqe_test.py | 1 - .../src/main/python/arithmetic_ops_test.py | 2 -- .../src/main/python/array_test.py | 25 +------------------ integration_tests/src/main/python/ast_test.py | 8 ------ .../src/main/python/cache_test.py | 4 --- .../src/main/python/cast_test.py | 5 ++-- integration_tests/src/main/python/cmp_test.py | 14 ----------- .../src/main/python/collection_ops_test.py | 10 -------- .../src/main/python/conditionals_test.py | 8 ------ integration_tests/src/main/python/csv_test.py | 1 - .../src/main/python/datasourcev2_read_test.py | 4 --- .../src/main/python/date_time_test.py | 3 ++- .../src/main/python/expand_exec_test.py | 1 - .../python/fastparquet_compatibility_test.py | 3 --- .../src/main/python/generate_expr_test.py | 7 ------ .../src/main/python/hash_aggregate_test.py | 2 -- .../src/main/python/hashing_test.py | 2 -- .../src/main/python/join_test.py | 2 -- .../src/main/python/limit_test.py | 2 -- integration_tests/src/main/python/map_test.py | 3 --- .../src/main/python/mortgage_test.py | 1 - .../src/main/python/qa_nightly_select_test.py | 4 --- .../src/main/python/repart_test.py | 4 --- .../src/main/python/row-based_udf_test.py | 2 -- .../src/main/python/row_conversion_test.py | 3 --- .../src/main/python/sample_test.py | 2 -- .../src/main/python/schema_evolution_test.py | 1 - .../src/main/python/sort_test.py | 7 ------ .../src/main/python/struct_test.py | 2 -- .../src/main/python/subquery_test.py | 3 --- .../src/main/python/time_window_test.py | 6 ----- integration_tests/src/main/python/udf_test.py | 5 ---- .../src/main/python/window_function_test.py | 4 --- 33 files changed, 6 insertions(+), 145 deletions(-) diff --git a/integration_tests/src/main/python/aqe_test.py b/integration_tests/src/main/python/aqe_test.py index 189bef329d7..e80e1a2db2b 100755 --- a/integration_tests/src/main/python/aqe_test.py +++ b/integration_tests/src/main/python/aqe_test.py @@ -196,7 +196,6 @@ def do_it(spark): @ignore_order(local=True) @allow_non_gpu('BroadcastNestedLoopJoinExec', 'Cast', 'DateSub', *db_113_cpu_bnlj_join_allow) @pytest.mark.parametrize('join', joins, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_aqe_join_reused_exchange_inequality_condition(spark_tmp_path, join): data_path = spark_tmp_path + '/PARQUET_DATA' def prep(spark): diff --git a/integration_tests/src/main/python/arithmetic_ops_test.py b/integration_tests/src/main/python/arithmetic_ops_test.py index e182a0433b9..3a207c43daf 100644 --- a/integration_tests/src/main/python/arithmetic_ops_test.py +++ b/integration_tests/src/main/python/arithmetic_ops_test.py @@ -988,7 +988,6 @@ def test_columnar_pow(data_gen): lambda spark : binary_op_df(spark, data_gen).selectExpr('pow(a, b)')) @pytest.mark.parametrize('data_gen', all_basic_gens + _arith_decimal_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_least(data_gen): num_cols = 20 s1 = with_cpu_session( @@ -1005,7 +1004,6 @@ def test_least(data_gen): f.least(*command_args))) @pytest.mark.parametrize('data_gen', all_basic_gens + _arith_decimal_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_greatest(data_gen): num_cols = 20 s1 = with_cpu_session( diff --git a/integration_tests/src/main/python/array_test.py b/integration_tests/src/main/python/array_test.py index 29f4e64b893..ec29dce70d1 100644 --- a/integration_tests/src/main/python/array_test.py +++ b/integration_tests/src/main/python/array_test.py @@ -16,7 +16,7 @@ from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_are_equal_sql, assert_gpu_and_cpu_error, assert_gpu_fallback_collect from data_gen import * -from conftest import is_databricks_runtime, is_not_utc +from conftest import is_databricks_runtime from marks import incompat from spark_session import is_before_spark_313, is_before_spark_330, is_databricks113_or_later, is_spark_330_or_later, is_databricks104_or_later, is_spark_33X, is_spark_340_or_later, is_spark_330, is_spark_330cdh from pyspark.sql.types import * @@ -103,13 +103,11 @@ @pytest.mark.parametrize('data_gen', array_item_test_gens, ids=idfn) @pytest.mark.parametrize('index_gen', array_index_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_item(data_gen, index_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: two_col_df(spark, data_gen, index_gen).selectExpr('a[b]')) @pytest.mark.parametrize('data_gen', array_item_test_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_item_lit_ordinal(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -147,7 +145,6 @@ def test_array_item_with_strict_index(strict_index_enabled, index): # No need to test this for multiple data types for array. Only one is enough, but with two kinds of invalid index. @pytest.mark.parametrize('index', [-2, 100, array_neg_index_gen, array_out_index_gen], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_item_ansi_fail_invalid_index(index): message = "SparkArrayIndexOutOfBoundsException" if (is_databricks104_or_later() or is_spark_330_or_later()) else "java.lang.ArrayIndexOutOfBoundsException" if isinstance(index, int): @@ -174,7 +171,6 @@ def test_array_item_ansi_not_fail_all_null_data(): decimal_gen_32bit, decimal_gen_64bit, decimal_gen_128bit, binary_gen, StructGen([['child0', StructGen([['child01', IntegerGen()]])], ['child1', string_gen], ['child2', float_gen]], nullable=False), StructGen([['child0', byte_gen], ['child1', string_gen], ['child2', float_gen]], nullable=False)], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_make_array(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars_for_sql(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -187,7 +183,6 @@ def test_make_array(data_gen): @pytest.mark.parametrize('data_gen', single_level_array_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_orderby_array_unique(data_gen): assert_gpu_and_cpu_are_equal_sql( lambda spark : append_unique_int_col_to_df(spark, unary_op_df(spark, data_gen)), @@ -217,7 +212,6 @@ def test_orderby_array_of_structs(data_gen): @pytest.mark.parametrize('data_gen', [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, date_gen, timestamp_gen], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_contains(data_gen): arr_gen = ArrayGen(data_gen) literal = with_cpu_session(lambda spark: gen_scalar(data_gen, force_no_nulls=True)) @@ -245,7 +239,6 @@ def test_array_contains_for_nans(data_gen): @pytest.mark.parametrize('data_gen', array_item_test_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_element_at(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: two_col_df(spark, data_gen, array_no_zero_index_gen).selectExpr( @@ -310,7 +303,6 @@ def test_array_element_at_zero_index_fail(index, ansi_enabled): @pytest.mark.parametrize('data_gen', array_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_transform(data_gen): def do_it(spark): columns = ['a', 'b', @@ -345,7 +337,6 @@ def do_it(spark): string_gen, boolean_gen, date_gen, timestamp_gen, null_gen] + decimal_gens @pytest.mark.parametrize('data_gen', array_min_max_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_min_max(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, ArrayGen(data_gen)).selectExpr( @@ -370,7 +361,6 @@ def test_array_concat_decimal(data_gen): 'concat(a, a)'))) @pytest.mark.parametrize('data_gen', orderable_gens + nested_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_repeat_with_count_column(data_gen): cnt_gen = IntegerGen(min_val=-5, max_val=5, special_cases=[]) cnt_not_null_gen = IntegerGen(min_val=-5, max_val=5, special_cases=[], nullable=False) @@ -384,7 +374,6 @@ def test_array_repeat_with_count_column(data_gen): @pytest.mark.parametrize('data_gen', orderable_gens + nested_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_repeat_with_count_scalar(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -414,7 +403,6 @@ def test_sql_array_scalars(query): @pytest.mark.parametrize('data_gen', all_basic_gens + nested_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_get_array_struct_fields(data_gen): array_struct_gen = ArrayGen( StructGen([['child0', data_gen], ['child1', int_gen]]), @@ -453,7 +441,6 @@ def do_it(spark): @pytest.mark.parametrize('data_gen', array_zips_gen, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_arrays_zip(data_gen): gen = StructGen( [('a', data_gen), ('b', data_gen), ('c', data_gen), ('d', data_gen)], nullable=False) @@ -486,7 +473,6 @@ def q1(spark): @incompat @pytest.mark.parametrize('data_gen', no_neg_zero_all_basic_gens + decimal_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') @pytest.mark.skipif(is_before_spark_313() or is_spark_330() or is_spark_330cdh(), reason="NaN equality is only handled in Spark 3.1.3+ and SPARK-39976 issue with null and ArrayIntersect in Spark 3.3.0") def test_array_intersect(data_gen): gen = StructGen( @@ -528,7 +514,6 @@ def test_array_intersect_spark330(data_gen): @incompat @pytest.mark.parametrize('data_gen', no_neg_zero_all_basic_gens_no_nans + decimal_gens, ids=idfn) @pytest.mark.skipif(not is_before_spark_313(), reason="NaN equality is only handled in Spark 3.1.3+") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_intersect_before_spark313(data_gen): gen = StructGen( [('a', ArrayGen(data_gen, nullable=True)), @@ -549,7 +534,6 @@ def test_array_intersect_before_spark313(data_gen): @incompat @pytest.mark.parametrize('data_gen', no_neg_zero_all_basic_gens + decimal_gens, ids=idfn) @pytest.mark.skipif(is_before_spark_313(), reason="NaN equality is only handled in Spark 3.1.3+") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_union(data_gen): gen = StructGen( [('a', ArrayGen(data_gen, nullable=True)), @@ -570,7 +554,6 @@ def test_array_union(data_gen): @incompat @pytest.mark.parametrize('data_gen', no_neg_zero_all_basic_gens_no_nans + decimal_gens, ids=idfn) @pytest.mark.skipif(not is_before_spark_313(), reason="NaN equality is only handled in Spark 3.1.3+") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_union_before_spark313(data_gen): gen = StructGen( [('a', ArrayGen(data_gen, nullable=True)), @@ -591,7 +574,6 @@ def test_array_union_before_spark313(data_gen): @incompat @pytest.mark.parametrize('data_gen', no_neg_zero_all_basic_gens + decimal_gens, ids=idfn) @pytest.mark.skipif(is_before_spark_313(), reason="NaN equality is only handled in Spark 3.1.3+") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_except(data_gen): gen = StructGen( [('a', ArrayGen(data_gen, nullable=True)), @@ -612,7 +594,6 @@ def test_array_except(data_gen): @incompat @pytest.mark.parametrize('data_gen', no_neg_zero_all_basic_gens_no_nans + decimal_gens, ids=idfn) @pytest.mark.skipif(not is_before_spark_313(), reason="NaN equality is only handled in Spark 3.1.3+") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_except_before_spark313(data_gen): gen = StructGen( [('a', ArrayGen(data_gen, nullable=True)), @@ -633,7 +614,6 @@ def test_array_except_before_spark313(data_gen): @incompat @pytest.mark.parametrize('data_gen', no_neg_zero_all_basic_gens + decimal_gens, ids=idfn) @pytest.mark.skipif(is_before_spark_313(), reason="NaN equality is only handled in Spark 3.1.3+") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_arrays_overlap(data_gen): gen = StructGen( [('a', ArrayGen(data_gen, nullable=True)), @@ -655,7 +635,6 @@ def test_arrays_overlap(data_gen): @incompat @pytest.mark.parametrize('data_gen', no_neg_zero_all_basic_gens_no_nans + decimal_gens, ids=idfn) @pytest.mark.skipif(not is_before_spark_313(), reason="NaN equality is only handled in Spark 3.1.3+") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_arrays_overlap_before_spark313(data_gen): gen = StructGen( [('a', ArrayGen(data_gen, nullable=True)), @@ -693,7 +672,6 @@ def test_array_remove_scalar(data_gen): FloatGen(special_cases=_non_neg_zero_float_special_cases + [-0.0]), DoubleGen(special_cases=_non_neg_zero_double_special_cases + [-0.0]), StringGen(pattern='[0-9]{1,5}'), boolean_gen, date_gen, timestamp_gen] + decimal_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_array_remove(data_gen): gen = StructGen( [('a', ArrayGen(data_gen, nullable=True)), @@ -708,7 +686,6 @@ def test_array_remove(data_gen): @pytest.mark.parametrize('data_gen', [ArrayGen(sub_gen) for sub_gen in array_gens_sample], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_flatten_array(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr('flatten(a)') diff --git a/integration_tests/src/main/python/ast_test.py b/integration_tests/src/main/python/ast_test.py index 2c06c51a876..a7af84180c5 100644 --- a/integration_tests/src/main/python/ast_test.py +++ b/integration_tests/src/main/python/ast_test.py @@ -71,7 +71,6 @@ def assert_binary_ast(data_descr, func, conf={}): assert_gpu_ast(is_supported, lambda spark: func(binary_op_df(spark, data_gen)), conf=conf) @pytest.mark.parametrize('data_gen', [boolean_gen, byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, timestamp_gen, date_gen], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_literal(spark_tmp_path, data_gen): # Write data to Parquet so Spark generates a plan using just the count of the data. data_path = spark_tmp_path + '/AST_TEST_DATA' @@ -81,7 +80,6 @@ def test_literal(spark_tmp_path, data_gen): func=lambda spark: spark.read.parquet(data_path).select(scalar)) @pytest.mark.parametrize('data_gen', [boolean_gen, byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, timestamp_gen, date_gen], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_null_literal(spark_tmp_path, data_gen): # Write data to Parquet so Spark generates a plan using just the count of the data. data_path = spark_tmp_path + '/AST_TEST_DATA' @@ -235,7 +233,6 @@ def test_expm1(data_descr): assert_unary_ast(data_descr, lambda df: df.selectExpr('expm1(a)')) @pytest.mark.parametrize('data_descr', ast_comparable_descrs, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_eq(data_descr): (s1, s2) = with_cpu_session(lambda spark: gen_scalars(data_descr[0], 2)) assert_binary_ast(data_descr, @@ -245,7 +242,6 @@ def test_eq(data_descr): f.col('a') == f.col('b'))) @pytest.mark.parametrize('data_descr', ast_comparable_descrs, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_ne(data_descr): (s1, s2) = with_cpu_session(lambda spark: gen_scalars(data_descr[0], 2)) assert_binary_ast(data_descr, @@ -255,7 +251,6 @@ def test_ne(data_descr): f.col('a') != f.col('b'))) @pytest.mark.parametrize('data_descr', ast_comparable_descrs, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_lt(data_descr): (s1, s2) = with_cpu_session(lambda spark: gen_scalars(data_descr[0], 2)) assert_binary_ast(data_descr, @@ -265,7 +260,6 @@ def test_lt(data_descr): f.col('a') < f.col('b'))) @pytest.mark.parametrize('data_descr', ast_comparable_descrs, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_lte(data_descr): (s1, s2) = with_cpu_session(lambda spark: gen_scalars(data_descr[0], 2)) assert_binary_ast(data_descr, @@ -275,7 +269,6 @@ def test_lte(data_descr): f.col('a') <= f.col('b'))) @pytest.mark.parametrize('data_descr', ast_comparable_descrs, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_gt(data_descr): (s1, s2) = with_cpu_session(lambda spark: gen_scalars(data_descr[0], 2)) assert_binary_ast(data_descr, @@ -285,7 +278,6 @@ def test_gt(data_descr): f.col('a') > f.col('b'))) @pytest.mark.parametrize('data_descr', ast_comparable_descrs, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_gte(data_descr): (s1, s2) = with_cpu_session(lambda spark: gen_scalars(data_descr[0], 2)) assert_binary_ast(data_descr, diff --git a/integration_tests/src/main/python/cache_test.py b/integration_tests/src/main/python/cache_test.py index e028e93a959..d2318a50c18 100644 --- a/integration_tests/src/main/python/cache_test.py +++ b/integration_tests/src/main/python/cache_test.py @@ -65,7 +65,6 @@ def test_passing_gpuExpr_as_Expr(enable_vectorized_conf): @pytest.mark.parametrize('data_gen', all_gen, ids=idfn) @pytest.mark.parametrize('enable_vectorized_conf', enable_vectorized_confs, ids=idfn) @ignore_order -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_cache_join(data_gen, enable_vectorized_conf): def do_join(spark): left, right = create_df(spark, data_gen, 500, 500) @@ -93,7 +92,6 @@ def do_join(spark): @pytest.mark.parametrize('data_gen', all_gen, ids=idfn) @pytest.mark.parametrize('enable_vectorized_conf', enable_vectorized_confs, ids=idfn) @ignore_order -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_cache_expand_exec(data_gen, enable_vectorized_conf): def op_df(spark, length=2048): cached = gen_df(spark, StructGen([ @@ -168,7 +166,6 @@ def n_fold(spark): @pytest.mark.parametrize('enable_vectorized', ['true', 'false'], ids=idfn) @ignore_order @allow_non_gpu("SortExec", "ShuffleExchangeExec", "RangePartitioning") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_cache_columnar(spark_tmp_path, data_gen, enable_vectorized, ts_write): data_path_gpu = spark_tmp_path + '/PARQUET_DATA' def read_parquet_cached(data_path): @@ -281,7 +278,6 @@ def helper(spark): @pytest.mark.parametrize('enable_vectorized_conf', enable_vectorized_confs, ids=idfn) @pytest.mark.parametrize('batch_size', [{"spark.rapids.sql.batchSizeBytes": "100"}, {}], ids=idfn) @ignore_order -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_cache_count(data_gen, with_x_session, enable_vectorized_conf, batch_size): test_conf = copy_and_update(enable_vectorized_conf, batch_size) generate_data_and_test_func_on_cached_df(with_x_session, lambda df: df.count(), data_gen, test_conf) diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index dbb41b60bb7..60b718ba15f 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -61,6 +61,7 @@ def test_cast_nested(data_gen, to_type): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).select(f.col('a').cast(to_type))) +@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') @datagen_overrides(seed=0, reason="https://github.com/NVIDIA/spark-rapids/issues/9781") def test_cast_string_date_valid_format(): # In Spark 3.2.0+ the valid format changed, and we cannot support all of the format. @@ -99,6 +100,7 @@ def test_cast_string_date_invalid_ansi_before_320(): 'spark.sql.ansi.enabled': 'true'}, ) # test Spark versions >= 320 and databricks, ANSI mode, valid values +@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') @pytest.mark.skipif(is_before_spark_320(), reason="Spark versions(< 320) not support Ansi mode when casting string to date") def test_cast_string_date_valid_ansi(): data_rows = [(v,) for v in valid_values_string_to_date] @@ -141,6 +143,7 @@ def test_try_cast_fallback_340(invalid): 'spark.sql.ansi.enabled': True}) # test all Spark versions, non ANSI mode, invalid value will be converted to NULL +@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_cast_string_date_non_ansi(): data_rows = [(v,) for v in values_string_to_data] assert_gpu_and_cpu_are_equal_collect( @@ -320,7 +323,6 @@ def test_cast_array_with_unmatched_element_to_string(data_gen, legacy): @pytest.mark.parametrize('data_gen', basic_map_gens_for_cast_to_string, ids=idfn) @pytest.mark.parametrize('legacy', ['true', 'false']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_cast_map_to_string(data_gen, legacy): _assert_cast_to_string_equal( data_gen, @@ -340,7 +342,6 @@ def test_cast_map_with_unmatched_element_to_string(data_gen, legacy): @pytest.mark.parametrize('data_gen', [StructGen([[str(i), gen] for i, gen in enumerate(basic_array_struct_gens_for_cast_to_string)] + [["map", MapGen(ByteGen(nullable=False), null_gen)]])], ids=idfn) @pytest.mark.parametrize('legacy', ['true', 'false']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_cast_struct_to_string(data_gen, legacy): _assert_cast_to_string_equal( data_gen, diff --git a/integration_tests/src/main/python/cmp_test.py b/integration_tests/src/main/python/cmp_test.py index a891b667016..b451f9c3db7 100644 --- a/integration_tests/src/main/python/cmp_test.py +++ b/integration_tests/src/main/python/cmp_test.py @@ -23,7 +23,6 @@ import pyspark.sql.functions as f @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + struct_gens_sample_with_decimal128_no_list, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_eq(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -57,7 +56,6 @@ def test_func(data_gen): test_func(data_gen) @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + struct_gens_sample_with_decimal128_no_list, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_eq_ns(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -85,7 +83,6 @@ def test_eq_ns_for_interval(): f.col('a').eqNullSafe(f.col('b')))) @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + struct_gens_sample_with_decimal128_no_list, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_ne(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -119,7 +116,6 @@ def test_func(data_gen): test_func(data_gen) @pytest.mark.parametrize('data_gen', orderable_gens + struct_gens_sample_with_decimal128_no_list, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_lt(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -153,7 +149,6 @@ def test_func(data_gen): test_func(data_gen) @pytest.mark.parametrize('data_gen', orderable_gens + struct_gens_sample_with_decimal128_no_list, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_lte(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -188,7 +183,6 @@ def test_func(data_gen): test_func(data_gen) @pytest.mark.parametrize('data_gen', orderable_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_gt(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -223,7 +217,6 @@ def test_func(data_gen): test_func(data_gen) @pytest.mark.parametrize('data_gen', orderable_gens + struct_gens_sample_with_decimal128_no_list, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_gte(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -258,7 +251,6 @@ def test_func(data_gen): test_func(data_gen) @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + [binary_gen] + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_isnull(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).select( @@ -278,27 +270,23 @@ def test_isnan(data_gen): f.isnan(f.col('a')))) @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + [binary_gen] + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_dropna_any(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).dropna()) @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + [binary_gen] + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_dropna_all(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).dropna(how='all')) #dropna is really a filter along with a test for null, but lets do an explicit filter test too @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_filter(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : three_col_df(spark, BooleanGen(), data_gen, data_gen).filter(f.col('a'))) # coalesce batch happens after a filter, but only if something else happens on the GPU after that @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_filter_with_project(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, BooleanGen(), data_gen).filter(f.col('a')).selectExpr('*', 'a as a2')) @@ -308,7 +296,6 @@ def test_filter_with_project(data_gen): # and some constants that then make it so all we need is the number of rows # of input. @pytest.mark.parametrize('op', ['>', '<']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_empty_filter(op, spark_tmp_path): def do_it(spark): @@ -337,7 +324,6 @@ def test_filter_with_lit(expr): # Spark supports two different versions of 'IN', and it depends on the spark.sql.optimizer.inSetConversionThreshold conf # This is to test entries under that value. @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_in(data_gen): # nulls are not supported for in on the GPU yet num_entries = int(with_cpu_session(lambda spark: spark.conf.get('spark.sql.optimizer.inSetConversionThreshold'))) - 1 diff --git a/integration_tests/src/main/python/collection_ops_test.py b/integration_tests/src/main/python/collection_ops_test.py index 43cc782df0f..04841e7f3ff 100644 --- a/integration_tests/src/main/python/collection_ops_test.py +++ b/integration_tests/src/main/python/collection_ops_test.py @@ -35,7 +35,6 @@ for sub_gen in all_gen + [null_gen]] @pytest.mark.parametrize('data_gen', non_nested_array_gens + nested_array_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_concat_list(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: three_col_df(spark, data_gen, data_gen, data_gen).selectExpr( @@ -46,7 +45,6 @@ def test_concat_list(data_gen): ) @pytest.mark.parametrize('dg', non_nested_array_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_concat_double_list_with_lit(dg): data_gen = ArrayGen(dg, max_length=2) array_lit = with_cpu_session(lambda spark: gen_scalar(data_gen)) @@ -70,7 +68,6 @@ def test_concat_double_list_with_lit(dg): @pytest.mark.parametrize('data_gen', non_nested_array_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_concat_list_with_lit(data_gen): lit_col1 = with_cpu_session(lambda spark: f.lit(gen_scalar(data_gen))).cast(data_gen.data_type) lit_col2 = with_cpu_session(lambda spark: f.lit(gen_scalar(data_gen))).cast(data_gen.data_type) @@ -99,7 +96,6 @@ def test_concat_string(): f.concat(f.col('a'), f.lit('')))) @pytest.mark.parametrize('data_gen', map_gens_sample + decimal_64_map_gens + decimal_128_map_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_map_concat(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: three_col_df(spark, data_gen, data_gen, data_gen @@ -111,7 +107,6 @@ def test_map_concat(data_gen): ) @pytest.mark.parametrize('data_gen', map_gens_sample + decimal_64_map_gens + decimal_128_map_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_map_concat_with_lit(data_gen): lit_col1 = with_cpu_session(lambda spark: f.lit(gen_scalar(data_gen))).cast(data_gen.data_type) lit_col2 = with_cpu_session(lambda spark: f.lit(gen_scalar(data_gen))).cast(data_gen.data_type) @@ -125,7 +120,6 @@ def test_map_concat_with_lit(data_gen): @pytest.mark.parametrize('data_gen', all_gen + nested_gens, ids=idfn) @pytest.mark.parametrize('size_of_null', ['true', 'false'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_size_of_array(data_gen, size_of_null): gen = ArrayGen(data_gen) assert_gpu_and_cpu_are_equal_collect( @@ -134,14 +128,12 @@ def test_size_of_array(data_gen, size_of_null): @pytest.mark.parametrize('data_gen', map_gens_sample, ids=idfn) @pytest.mark.parametrize('size_of_null', ['true', 'false'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_size_of_map(data_gen, size_of_null): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr('size(a)'), conf={'spark.sql.legacy.sizeOfNull': size_of_null}) @pytest.mark.parametrize('data_gen', array_gens_sample + [string_gen], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_reverse(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr('reverse(a)')) @@ -152,7 +144,6 @@ def test_reverse(data_gen): ] @pytest.mark.parametrize('data_gen', _sort_array_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_sort_array(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).select( @@ -160,7 +151,6 @@ def test_sort_array(data_gen): f.sort_array(f.col('a'), False))) @pytest.mark.parametrize('data_gen', _sort_array_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_sort_array_lit(data_gen): array_lit = with_cpu_session(lambda spark: gen_scalar(data_gen)) assert_gpu_and_cpu_are_equal_collect( diff --git a/integration_tests/src/main/python/conditionals_test.py b/integration_tests/src/main/python/conditionals_test.py index 48d5a05c099..1bc11c5d27b 100644 --- a/integration_tests/src/main/python/conditionals_test.py +++ b/integration_tests/src/main/python/conditionals_test.py @@ -45,7 +45,6 @@ def mk_str_gen(pattern): if_nested_gens = if_array_gens_sample + if_struct_gens_sample @pytest.mark.parametrize('data_gen', all_gens + if_nested_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_if_else(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars_for_sql(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -65,7 +64,6 @@ def test_if_else(data_gen): # Maps scalars are not really supported by Spark from python without jumping through a lot of hoops # so for now we are going to skip them @pytest.mark.parametrize('data_gen', map_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_if_else_map(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : three_col_df(spark, boolean_gen, data_gen, data_gen).selectExpr( @@ -75,7 +73,6 @@ def test_if_else_map(data_gen): @datagen_overrides(seed=0, reason='https://github.com/NVIDIA/spark-rapids/issues/9685') @pytest.mark.order(1) # at the head of xdist worker queue if pytest-order is installed @pytest.mark.parametrize('data_gen', all_gens + all_nested_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_case_when(data_gen): num_cmps = 20 s1 = with_cpu_session( @@ -119,7 +116,6 @@ def test_nanvl(data_gen): f.nanvl(f.lit(float('nan')).cast(data_type), f.col('b')))) @pytest.mark.parametrize('data_gen', all_basic_gens + decimal_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_nvl(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars_for_sql(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -139,7 +135,6 @@ def test_nvl(data_gen): # at least one `BoundReference` @datagen_overrides(seed=0, reason='https://github.com/NVIDIA/spark-rapids/issues/9684') @pytest.mark.parametrize('data_gen', all_gens + all_nested_gens_nonempty_struct + map_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_coalesce(data_gen): num_cols = 20 s1 = with_cpu_session( @@ -161,7 +156,6 @@ def test_coalesce_constant_output(): lambda spark : spark.range(1, 100).selectExpr("4 + coalesce(5, id) as nine")) @pytest.mark.parametrize('data_gen', all_basic_gens + decimal_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_nvl2(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars_for_sql(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -175,7 +169,6 @@ def test_nvl2(data_gen): 'nvl2(a, {}, c)'.format(null_lit))) @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_nullif(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars_for_sql(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -189,7 +182,6 @@ def test_nullif(data_gen): 'nullif(a, {})'.format(null_lit))) @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_ifnull(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars_for_sql(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index c10221a4407..02c12e333e0 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -248,7 +248,6 @@ def read_impl(spark): @pytest.mark.parametrize('read_func', [read_csv_df, read_csv_sql]) @pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_basic_csv_read(std_input_path, name, schema, options, read_func, v1_enabled_list, ansi_enabled, spark_tmp_table_factory): updated_conf=copy_and_update(_enable_all_types_conf, { 'spark.sql.sources.useV1SourceList': v1_enabled_list, diff --git a/integration_tests/src/main/python/datasourcev2_read_test.py b/integration_tests/src/main/python/datasourcev2_read_test.py index cc141700cb8..4a25d618e7d 100644 --- a/integration_tests/src/main/python/datasourcev2_read_test.py +++ b/integration_tests/src/main/python/datasourcev2_read_test.py @@ -27,7 +27,6 @@ def readTable(types, classToUse): .orderBy("col1") @validate_execs_in_gpu_plan('HostColumnarToGpu') -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_read_int(): assert_gpu_and_cpu_are_equal_collect(readTable("int", columnarClass)) @@ -37,21 +36,18 @@ def test_read_strings(): assert_gpu_and_cpu_are_equal_collect(readTable("string", columnarClass)) @validate_execs_in_gpu_plan('HostColumnarToGpu') -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_read_all_types(): assert_gpu_and_cpu_are_equal_collect( readTable("int,bool,byte,short,long,string,float,double,date,timestamp", columnarClass), conf={'spark.rapids.sql.castFloatToString.enabled': 'true'}) @validate_execs_in_gpu_plan('HostColumnarToGpu') -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_read_all_types_count(): assert_gpu_and_cpu_row_counts_equal( readTable("int,bool,byte,short,long,string,float,double,date,timestamp", columnarClass), conf={'spark.rapids.sql.castFloatToString.enabled': 'true'}) @validate_execs_in_gpu_plan('HostColumnarToGpu') -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_read_arrow_off(): assert_gpu_and_cpu_are_equal_collect( readTable("int,bool,byte,short,long,string,float,double,date,timestamp", columnarClass), diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index 1d984193f9e..a731952ed2c 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -25,6 +25,7 @@ # We only support literal intervals for TimeSub vals = [(-584, 1563), (1943, 1101), (2693, 2167), (2729, 0), (44, 1534), (2635, 3319), (1885, -2828), (0, 2463), (932, 2286), (0, 0)] + @pytest.mark.parametrize('data_gen', vals, ids=idfn) @pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_timesub(data_gen): @@ -63,7 +64,6 @@ def test_interval_seconds_overflow_exception(): error_message="IllegalArgumentException") @pytest.mark.parametrize('data_gen', vals, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_timeadd_from_subquery(data_gen): def fun(spark): @@ -458,6 +458,7 @@ def test_gettimestamp_ansi_exception(): supported_date_formats = ['yyyy-MM-dd', 'yyyy-MM', 'yyyy/MM/dd', 'yyyy/MM', 'dd/MM/yyyy', 'MM-dd', 'MM/dd', 'dd-MM', 'dd/MM'] + @pytest.mark.parametrize('date_format', supported_date_formats, ids=idfn) @pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) @pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') diff --git a/integration_tests/src/main/python/expand_exec_test.py b/integration_tests/src/main/python/expand_exec_test.py index abb9a7bd094..5a8be470a44 100644 --- a/integration_tests/src/main/python/expand_exec_test.py +++ b/integration_tests/src/main/python/expand_exec_test.py @@ -23,7 +23,6 @@ # Many Spark versions have issues sorting large decimals, # see https://issues.apache.org/jira/browse/SPARK-40089. @ignore_order(local=True) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_expand_exec(data_gen): def op_df(spark, length=2048): return gen_df(spark, StructGen([ diff --git a/integration_tests/src/main/python/fastparquet_compatibility_test.py b/integration_tests/src/main/python/fastparquet_compatibility_test.py index a12bd223778..86cac2aefed 100644 --- a/integration_tests/src/main/python/fastparquet_compatibility_test.py +++ b/integration_tests/src/main/python/fastparquet_compatibility_test.py @@ -142,7 +142,6 @@ def read_with_fastparquet_or_plugin(spark): marks=pytest.mark.xfail(is_databricks_runtime(), reason="https://github.com/NVIDIA/spark-rapids/issues/9778")), ], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_reading_file_written_by_spark_cpu(data_gen, spark_tmp_path): """ This test writes data_gen output to Parquet via Apache Spark, then verifies that fastparquet and the RAPIDS @@ -209,7 +208,6 @@ def test_reading_file_written_by_spark_cpu(data_gen, spark_tmp_path): end=pandas_min_datetime), marks=pytest.mark.xfail(reason="fastparquet reads timestamps preceding 1900 incorrectly.")), ], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_reading_file_written_with_gpu(spark_tmp_path, column_gen): """ This test writes the data-gen output to file via the RAPIDS plugin, then checks that the data is read identically @@ -392,7 +390,6 @@ def write_with_fastparquet(spark, data_gen): marks=pytest.mark.xfail(reason="fastparquet fails to read nullable Struct columns written from Apache Spark. " "It fails the rewrite to parquet, thereby failing the test.")), ], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_reading_file_rewritten_with_fastparquet(column_gen, time_format, spark_tmp_path): """ This test is a workaround to test data-types that have problems being converted diff --git a/integration_tests/src/main/python/generate_expr_test.py b/integration_tests/src/main/python/generate_expr_test.py index cde16352236..acc3e125ee6 100644 --- a/integration_tests/src/main/python/generate_expr_test.py +++ b/integration_tests/src/main/python/generate_expr_test.py @@ -38,7 +38,6 @@ def four_op_df(spark, gen, length=2048): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_explode_makearray(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : four_op_df(spark, data_gen).selectExpr('a', 'explode(array(b, c, d))')) @@ -47,7 +46,6 @@ def test_explode_makearray(data_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_explode_litarray(data_gen): array_lit = with_cpu_session( lambda spark: gen_scalar(ArrayGen(data_gen, min_length=3, max_length=3, nullable=False))) @@ -63,7 +61,6 @@ def test_explode_litarray(data_gen): @pytest.mark.parametrize('data_gen', explode_gens + struct_gens_sample_with_decimal128 + array_gens_sample + map_gens_sample + arrays_with_binary + maps_with_binary, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_explode_array_data(data_gen): data_gen = [int_gen, ArrayGen(data_gen)] assert_gpu_and_cpu_are_equal_collect( @@ -74,7 +71,6 @@ def test_explode_array_data(data_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('map_gen', map_gens_sample + decimal_128_map_gens + maps_with_binary, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_explode_map_data(map_gen): data_gen = [int_gen, map_gen] assert_gpu_and_cpu_are_equal_collect( @@ -85,7 +81,6 @@ def test_explode_map_data(map_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_explode_nested_array_data(data_gen): data_gen = [int_gen, ArrayGen(ArrayGen(data_gen))] assert_gpu_and_cpu_are_equal_collect( @@ -100,7 +95,6 @@ def test_explode_nested_array_data(data_gen): @pytest.mark.parametrize('data_gen', explode_gens + struct_gens_sample_with_decimal128 + array_gens_sample + arrays_with_binary + map_gens_sample + maps_with_binary, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_explode_outer_array_data(data_gen): data_gen = [int_gen, ArrayGen(data_gen)] assert_gpu_and_cpu_are_equal_collect( @@ -111,7 +105,6 @@ def test_explode_outer_array_data(data_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('map_gen', map_gens_sample + decimal_128_map_gens + maps_with_binary, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_explode_outer_map_data(map_gen): data_gen = [int_gen, map_gen] assert_gpu_and_cpu_are_equal_collect( diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 0c99fc4516a..03422e3f4bc 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -334,7 +334,6 @@ def test_hash_grpby_sum_count_action(data_gen, override_split_until_size, overri @allow_non_gpu("SortAggregateExec", "SortExec", "ShuffleExchangeExec") @ignore_order @pytest.mark.parametrize('data_gen', _grpkey_nested_structs_with_array_basic_child + _grpkey_list_with_non_nested_children, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_hash_grpby_list_min_max(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=100).coalesce(1).groupby('a').agg(f.min('b'), f.max('b')) @@ -617,7 +616,6 @@ def test_decimal128_min_max_group_by(data_gen): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _all_basic_gens_with_all_nans_cases, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_min_max_group_by(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: two_col_df(spark, byte_gen, data_gen) diff --git a/integration_tests/src/main/python/hashing_test.py b/integration_tests/src/main/python/hashing_test.py index e2a753ecaeb..93a7f30e756 100644 --- a/integration_tests/src/main/python/hashing_test.py +++ b/integration_tests/src/main/python/hashing_test.py @@ -47,13 +47,11 @@ @ignore_order(local=True) @pytest.mark.parametrize("gen", _xxhash_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_xxhash64_single_column(gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, gen).selectExpr("a", "xxhash64(a)")) @ignore_order(local=True) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_xxhash64_multi_column(): gen = StructGen(_struct_of_xxhash_gens.children, nullable=False) col_list = ",".join(gen.data_type.fieldNames()) diff --git a/integration_tests/src/main/python/join_test.py b/integration_tests/src/main/python/join_test.py index ba172715936..9ea78a85401 100644 --- a/integration_tests/src/main/python/join_test.py +++ b/integration_tests/src/main/python/join_test.py @@ -170,7 +170,6 @@ def do_join(spark): (all_gen, '1g'), (join_small_batch_gens, '1000')), ids=idfn) @pytest.mark.parametrize('join_type', all_join_types, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_sortmerge_join(data_gen, join_type, batch_size): def do_join(spark): left, right = create_df(spark, data_gen, 500, 500) @@ -181,7 +180,6 @@ def do_join(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', basic_nested_gens + [decimal_gen_128bit], ids=idfn) @pytest.mark.parametrize('join_type', all_join_types, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_sortmerge_join_ridealong(data_gen, join_type): def do_join(spark): left, right = create_ridealong_df(spark, short_gen, data_gen, 500, 500) diff --git a/integration_tests/src/main/python/limit_test.py b/integration_tests/src/main/python/limit_test.py index efe81c1058a..123b40785fe 100644 --- a/integration_tests/src/main/python/limit_test.py +++ b/integration_tests/src/main/python/limit_test.py @@ -21,7 +21,6 @@ from marks import allow_non_gpu, approximate_float @pytest.mark.parametrize('data_gen', all_basic_gens + decimal_gens + array_gens_sample + map_gens_sample + struct_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_simple_limit(data_gen): assert_gpu_and_cpu_are_equal_collect( # We need some processing after the limit to avoid a CollectLimitExec @@ -82,7 +81,6 @@ def test_non_zero_offset_with_limit(limit, offset, batch_size): @pytest.mark.skipif(is_before_spark_340(), reason='offset is introduced from Spark 3.4.0') @allow_non_gpu('ShuffleExchangeExec') # when limit = 0, ShuffleExchangeExec is not replaced. @approximate_float -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_order_by_offset_with_limit(limit, offset, data_gen, batch_size): # In CPU version of spark, (limit, offset) can not be negative number. # Test case description: diff --git a/integration_tests/src/main/python/map_test.py b/integration_tests/src/main/python/map_test.py index 5daeb916e22..dbd02884ddc 100644 --- a/integration_tests/src/main/python/map_test.py +++ b/integration_tests/src/main/python/map_test.py @@ -81,7 +81,6 @@ def test_map_values(data_gen): @pytest.mark.parametrize('data_gen', supported_key_map_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_map_entries(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -117,7 +116,6 @@ def decimal_value_gen(): [MapGen(StringGen(pattern='key_[0-9]', nullable=False), value(), max_length=6) for value in get_map_value_gens()], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_get_map_value_string_keys(data_gen): index_gen = StringGen() assert_gpu_and_cpu_are_equal_collect( @@ -141,7 +139,6 @@ def test_get_map_value_string_keys(data_gen): @pytest.mark.parametrize('data_gen', numeric_key_map_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_get_map_value_numeric_keys(data_gen): key_gen = data_gen._key_gen assert_gpu_and_cpu_are_equal_collect( diff --git a/integration_tests/src/main/python/mortgage_test.py b/integration_tests/src/main/python/mortgage_test.py index 00bab066651..a6b283e7da8 100644 --- a/integration_tests/src/main/python/mortgage_test.py +++ b/integration_tests/src/main/python/mortgage_test.py @@ -23,7 +23,6 @@ @limit @ignore_order @allow_non_gpu(any=True) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_mortgage(mortgage): assert_gpu_and_cpu_are_equal_iterator( lambda spark : mortgage.do_test_query(spark)) diff --git a/integration_tests/src/main/python/qa_nightly_select_test.py b/integration_tests/src/main/python/qa_nightly_select_test.py index 1349de3fcdf..d1a894e8e00 100644 --- a/integration_tests/src/main/python/qa_nightly_select_test.py +++ b/integration_tests/src/main/python/qa_nightly_select_test.py @@ -185,7 +185,6 @@ def test_needs_sort_select(sql_query_line, pytestconfig): @ignore_order(local=True) @qarun @pytest.mark.parametrize('sql_query_line', SELECT_JOIN_SQL, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_select_join(sql_query_line, pytestconfig): sql_query = sql_query_line[0] if sql_query: @@ -202,7 +201,6 @@ def init_tables(spark): @ignore_order(local=True) @qarun @pytest.mark.parametrize('sql_query_line', SELECT_PRE_ORDER_SQL, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_select_first_last(sql_query_line, pytestconfig): sql_query = sql_query_line[0] if sql_query: @@ -215,7 +213,6 @@ def test_select_first_last(sql_query_line, pytestconfig): @ignore_order(local=True) @qarun @pytest.mark.parametrize('sql_query_line', SELECT_FLOAT_SQL, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_select_float_order_local(sql_query_line, pytestconfig): sql_query = sql_query_line[0] if sql_query: @@ -230,7 +227,6 @@ def test_select_float_order_local(sql_query_line, pytestconfig): @qarun @pytest.mark.parametrize('sql_query_line', SELECT_REGEXP_SQL, ids=idfn) @pytest.mark.skipif(not is_jvm_charset_utf8(), reason="Regular expressions require UTF-8") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_select_regexp(sql_query_line, pytestconfig): sql_query = sql_query_line[0] if sql_query: diff --git a/integration_tests/src/main/python/repart_test.py b/integration_tests/src/main/python/repart_test.py index 60e0a191f25..28782e77ceb 100644 --- a/integration_tests/src/main/python/repart_test.py +++ b/integration_tests/src/main/python/repart_test.py @@ -90,7 +90,6 @@ def test_union_struct_missing_children(data_gen): nested_struct, struct_of_maps], ids=idfn) # This tests union of two DFs of two cols each. The types of the left col and right col is the same -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_union(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).union(binary_op_df(spark, data_gen))) @@ -101,7 +100,6 @@ def test_union(data_gen): nested_struct, struct_of_maps], ids=idfn) # This tests union of two DFs of two cols each. The types of the left col and right col is the same -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_unionAll(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).unionAll(binary_op_df(spark, data_gen))) @@ -116,7 +114,6 @@ def test_unionAll(data_gen): struct_of_maps], ids=idfn) # This tests the union of two DFs of structs with missing child column names. The missing child # column will be replaced by nulls in the output DF. This is a feature added in 3.1+ -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_union_by_missing_col_name(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).withColumnRenamed("a", "x") @@ -158,7 +155,6 @@ def assert_union_equal(gen1, gen2): StructGen([['child0', DecimalGen(7, 2)]]), nested_struct, struct_of_maps], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_union_by_name(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).unionByName(binary_op_df(spark, data_gen))) diff --git a/integration_tests/src/main/python/row-based_udf_test.py b/integration_tests/src/main/python/row-based_udf_test.py index 19b02f2e24e..118110ba1c8 100644 --- a/integration_tests/src/main/python/row-based_udf_test.py +++ b/integration_tests/src/main/python/row-based_udf_test.py @@ -34,7 +34,6 @@ def load_hive_udf(spark, udfname, udfclass): @pytest.mark.xfail(condition=is_spark_350_or_later(), reason='https://github.com/NVIDIA/spark-rapids/issues/9064') -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_hive_empty_simple_udf(): with_spark_session(skip_if_no_hive) @@ -48,7 +47,6 @@ def evalfn(spark): "SELECT i, emptysimple(s, 'const_string') FROM hive_simple_udf_test_table", conf={'spark.rapids.sql.rowBasedUDF.enabled': 'true'}) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_hive_empty_generic_udf(): with_spark_session(skip_if_no_hive) def evalfn(spark): diff --git a/integration_tests/src/main/python/row_conversion_test.py b/integration_tests/src/main/python/row_conversion_test.py index bc13419c8ec..2a559d28cba 100644 --- a/integration_tests/src/main/python/row_conversion_test.py +++ b/integration_tests/src/main/python/row_conversion_test.py @@ -29,7 +29,6 @@ # to be brought back to the CPU (rows) to be returned. # So we just need a very simple operation in the middle that # can be done on the GPU. -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_row_conversions(): gens = [["a", byte_gen], ["b", short_gen], ["c", int_gen], ["d", long_gen], ["e", float_gen], ["f", double_gen], ["g", string_gen], ["h", boolean_gen], @@ -44,7 +43,6 @@ def test_row_conversions(): assert_gpu_and_cpu_are_equal_collect( lambda spark : gen_df(spark, gens).selectExpr("*", "a as a_again")) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_row_conversions_fixed_width(): gens = [["a", byte_gen], ["b", short_gen], ["c", int_gen], ["d", long_gen], ["e", float_gen], ["f", double_gen], ["h", boolean_gen], @@ -53,7 +51,6 @@ def test_row_conversions_fixed_width(): assert_gpu_and_cpu_are_equal_collect( lambda spark : gen_df(spark, gens).selectExpr("*", "a as a_again")) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_row_conversions_fixed_width_wide(): gens = [["a{}".format(i), ByteGen(nullable=True)] for i in range(10)] + \ [["b{}".format(i), ShortGen(nullable=True)] for i in range(10)] + \ diff --git a/integration_tests/src/main/python/sample_test.py b/integration_tests/src/main/python/sample_test.py index 5ae72212bed..d1769ccef03 100644 --- a/integration_tests/src/main/python/sample_test.py +++ b/integration_tests/src/main/python/sample_test.py @@ -39,7 +39,6 @@ def test_sample_produce_empty_batch(data_gen): # the following cases is the normal cases and do not use @ignore_order nested_gens = array_gens_sample + struct_gens_sample + map_gens_sample @pytest.mark.parametrize('data_gen', basic_gens + nested_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_sample(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen, num_slices = 10) @@ -47,7 +46,6 @@ def test_sample(data_gen): ) @pytest.mark.parametrize('data_gen', basic_gens + nested_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_sample_with_replacement(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen, num_slices = 10).sample( diff --git a/integration_tests/src/main/python/schema_evolution_test.py b/integration_tests/src/main/python/schema_evolution_test.py index d9f4c0f0899..f9766a80eef 100644 --- a/integration_tests/src/main/python/schema_evolution_test.py +++ b/integration_tests/src/main/python/schema_evolution_test.py @@ -62,7 +62,6 @@ def get_ddl(col_gen_pairs): @ignore_order(local=True) @pytest.mark.parametrize("format", _formats) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_column_add_after_partition(spark_tmp_table_factory, format): # Databricks 10.4 appears to be missing https://issues.apache.org/jira/browse/SPARK-39417 # so avoid generating nulls for numeric partitions diff --git a/integration_tests/src/main/python/sort_test.py b/integration_tests/src/main/python/sort_test.py index 7fe208ae12d..7a8d1c58612 100644 --- a/integration_tests/src/main/python/sort_test.py +++ b/integration_tests/src/main/python/sort_test.py @@ -52,7 +52,6 @@ def test_sort_nonbinary_carry_binary(data_gen): @pytest.mark.parametrize('data_gen', orderable_gens + orderable_not_null_gen, ids=idfn) @pytest.mark.parametrize('order', [f.col('a').asc(), f.col('a').asc_nulls_last(), f.col('a').desc(), f.col('a').desc_nulls_first()], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_single_orderby(data_gen, order): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).orderBy(order)) @@ -60,7 +59,6 @@ def test_single_orderby(data_gen, order): @pytest.mark.parametrize('data_gen', single_level_array_gens, ids=idfn) @pytest.mark.parametrize('order', [f.col('a').asc(), f.col('a').asc_nulls_first(), f.col('a').asc_nulls_last(), f.col('a').desc(), f.col('a').desc_nulls_first(), f.col('a').desc_nulls_last()], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_single_orderby_on_array(data_gen, order): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).orderBy(order)) @@ -105,7 +103,6 @@ def test_single_orderby_fallback_for_array_of_struct(data_gen, order): marks=pytest.mark.xfail(reason='opposite null order not supported')), pytest.param(f.col('a').desc_nulls_last()), ], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_single_nested_orderby_plain(data_gen, order, shuffle_parts, stable_sort): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).orderBy(order), @@ -133,7 +130,6 @@ def test_single_nested_orderby_fallback_for_nullorder(data_gen, order): orderable_without_neg_decimal = [n for n in (orderable_gens + orderable_not_null_gen) if not (isinstance(n, DecimalGen) and n.scale < 0)] @pytest.mark.parametrize('data_gen', orderable_without_neg_decimal + single_level_array_gens, ids=idfn) @pytest.mark.parametrize('order', [f.col('a').asc(), f.col('a').asc_nulls_last(), f.col('a').desc(), f.col('a').desc_nulls_first()], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_single_orderby_with_limit(data_gen, order): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).orderBy(order).limit(100)) @@ -144,7 +140,6 @@ def test_single_orderby_with_limit(data_gen, order): pytest.param(f.col('a').desc(), all_basic_struct_gen), pytest.param(f.col('a').desc_nulls_last(), all_basic_struct_gen) ], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_single_nested_orderby_with_limit(data_gen, order): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).orderBy(order).limit(100), @@ -167,7 +162,6 @@ def test_single_nested_orderby_with_limit_fallback(data_gen, order): @pytest.mark.parametrize('data_gen', orderable_gens + orderable_not_null_gen + single_level_array_gens, ids=idfn) @pytest.mark.parametrize('order', [f.col('a').asc(), f.col('a').asc_nulls_last(), f.col('a').desc(), f.col('a').desc_nulls_first()], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_single_sort_in_part(data_gen, order): # We set `num_slices` to handle https://github.com/NVIDIA/spark-rapids/issues/2477 assert_gpu_and_cpu_are_equal_collect( @@ -190,7 +184,6 @@ def test_single_sort_in_part(data_gen, order): pytest.param(f.col('a').desc_nulls_last()), ], ids=idfn) @pytest.mark.parametrize('stable_sort', ['STABLE', 'OUTOFCORE'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_single_nested_sort_in_part(data_gen, order, stable_sort): sort_conf = {'spark.rapids.sql.stableSort.enabled': stable_sort == 'STABLE'} assert_gpu_and_cpu_are_equal_collect( diff --git a/integration_tests/src/main/python/struct_test.py b/integration_tests/src/main/python/struct_test.py index 986781c32e0..18464089d9c 100644 --- a/integration_tests/src/main/python/struct_test.py +++ b/integration_tests/src/main/python/struct_test.py @@ -34,7 +34,6 @@ def test_struct_scalar_project(): StructGen([["first", decimal_gen_64bit], ["second", decimal_gen_32bit], ["third", decimal_gen_32bit]]), StructGen([["first", decimal_gen_128bit], ["second", decimal_gen_128bit], ["third", decimal_gen_128bit]]), StructGen([["first", binary_gen], ["second", ArrayGen(BinaryGen(max_length=10), max_length=10)], ["third", binary_gen]])], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_struct_get_item(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr( @@ -45,7 +44,6 @@ def test_struct_get_item(data_gen): @pytest.mark.parametrize('data_gen', all_basic_gens + decimal_gens + [binary_gen, null_gen] + single_level_array_gens + struct_gens_sample + map_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_make_struct(data_gen): # Spark has no good way to create a map literal without the map function # so we are inserting one. diff --git a/integration_tests/src/main/python/subquery_test.py b/integration_tests/src/main/python/subquery_test.py index 25a70b47a17..c58c81caf1d 100644 --- a/integration_tests/src/main/python/subquery_test.py +++ b/integration_tests/src/main/python/subquery_test.py @@ -20,7 +20,6 @@ @ignore_order(local=True) @pytest.mark.parametrize('data_gen', all_basic_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_scalar_subquery_basics(data_gen): # Fix num_slices at 1 to make sure that first/last returns same results under CPU and GPU. assert_gpu_and_cpu_are_equal_sql( @@ -33,7 +32,6 @@ def test_scalar_subquery_basics(data_gen): @ignore_order(local=True) @pytest.mark.parametrize('basic_gen', all_basic_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_scalar_subquery_struct(basic_gen): # single-level struct gen = [('ss', StructGen([['a', basic_gen], ['b', basic_gen]]))] @@ -68,7 +66,6 @@ def test_scalar_subquery_struct(basic_gen): @ignore_order(local=True) @pytest.mark.parametrize('basic_gen', all_basic_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_scalar_subquery_array(basic_gen): # single-level array assert_gpu_and_cpu_are_equal_sql( diff --git a/integration_tests/src/main/python/time_window_test.py b/integration_tests/src/main/python/time_window_test.py index 52071926309..318299742c1 100644 --- a/integration_tests/src/main/python/time_window_test.py +++ b/integration_tests/src/main/python/time_window_test.py @@ -30,7 +30,6 @@ @pytest.mark.parametrize('data_gen', integral_gens + [string_gen], ids=idfn) @ignore_order -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_grouped_tumbling_window(data_gen): row_gen = StructGen([['ts', _restricted_ts_gen],['data', data_gen]], nullable=False) assert_gpu_and_cpu_are_equal_collect( @@ -42,7 +41,6 @@ def test_grouped_tumbling_window(data_gen): @pytest.mark.parametrize('data_gen', integral_gens + [string_gen], ids=idfn) @ignore_order -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_grouped_sliding_window(data_gen): row_gen = StructGen([['ts', _restricted_ts_gen],['data', data_gen]], nullable=False) assert_gpu_and_cpu_are_equal_collect( @@ -50,7 +48,6 @@ def test_grouped_sliding_window(data_gen): @pytest.mark.parametrize('data_gen', integral_gens + [string_gen], ids=idfn) @ignore_order -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_grouped_sliding_window_array(data_gen): row_gen = StructGen([['ts', _restricted_ts_gen],['data', ArrayGen(data_gen)]], nullable=False) assert_gpu_and_cpu_are_equal_collect( @@ -58,7 +55,6 @@ def test_grouped_sliding_window_array(data_gen): @pytest.mark.parametrize('data_gen', integral_gens + [string_gen], ids=idfn) @ignore_order -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_tumbling_window(data_gen): row_gen = StructGen([['ts', _restricted_ts_gen],['data', data_gen]], nullable=False) w = Window.partitionBy(f.window('ts', '5 hour')) @@ -67,7 +63,6 @@ def test_tumbling_window(data_gen): @pytest.mark.parametrize('data_gen', integral_gens + [string_gen], ids=idfn) @ignore_order -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_sliding_window(data_gen): row_gen = StructGen([['ts', _restricted_ts_gen],['data', data_gen]], nullable=False) w = Window.partitionBy(f.window('ts', '5 hour', '1 hour')) @@ -78,7 +73,6 @@ def test_sliding_window(data_gen): @pytest.mark.parametrize('data_gen', all_basic_gens + decimal_gens + array_gens_sample + map_gens_sample, ids=idfn) # This includes an expand and we produce a different order than the CPU does. Sort locally to allow sorting of all types @ignore_order(local=True) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_just_window(data_gen): row_gen = StructGen([['ts', timestamp_gen],['data', data_gen]], nullable=False) assert_gpu_and_cpu_are_equal_collect( diff --git a/integration_tests/src/main/python/udf_test.py b/integration_tests/src/main/python/udf_test.py index 88281279162..1f530e5bb47 100644 --- a/integration_tests/src/main/python/udf_test.py +++ b/integration_tests/src/main/python/udf_test.py @@ -90,7 +90,6 @@ def iterator_add(to_process: Iterator[Tuple[pd.Series, pd.Series]]) -> Iterator[ @pytest.mark.parametrize('data_gen', data_gens_nested_for_udf, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_pandas_scalar_udf_nested_type(data_gen): def nested_size(nested): return pd.Series([nested.size]).repeat(len(nested)) @@ -117,7 +116,6 @@ def pandas_sum(to_process: pd.Series) -> float: @approximate_float @pytest.mark.parametrize('data_gen', arrow_common_gen, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_single_aggregate_udf_more_types(data_gen): @f.pandas_udf('double') def group_size_udf(to_process: pd.Series) -> float: @@ -148,7 +146,6 @@ def pandas_sum(to_process: pd.Series) -> int: @ignore_order(local=True) @pytest.mark.parametrize('data_gen', arrow_common_gen, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_group_aggregate_udf_more_types(data_gen): @f.pandas_udf('long') def group_size_udf(to_process: pd.Series) -> int: @@ -264,7 +261,6 @@ def pandas_add(data): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', arrow_common_gen, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_group_apply_udf_more_types(data_gen): def group_size_udf(key, pdf): return pd.DataFrame([[len(key), len(pdf), len(pdf.columns)]]) @@ -292,7 +288,6 @@ def pandas_filter(iterator): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', data_gens_nested_for_udf, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_pandas_map_udf_nested_type(data_gen): # Supported UDF output types by plugin: (commonCudfTypes + ARRAY).nested() + STRUCT # STRUCT represents the whole dataframe in Map Pandas UDF, so no struct column in UDF output. diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index d850403d118..81ce8d74948 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -451,7 +451,6 @@ def test_range_windows_with_string_order_by_column(data_gen, batch_size): # the order returned should be consistent because the data ends up in a single task (no partitioning) @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches @pytest.mark.parametrize('b_gen', all_basic_gens + [decimal_gen_32bit, decimal_gen_128bit], ids=meta_idfn('data:')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_window_batched_unbounded_no_part(b_gen, batch_size): conf = {'spark.rapids.sql.batchSizeBytes': batch_size, 'spark.rapids.sql.castFloatToDecimal.enabled': True} @@ -469,7 +468,6 @@ def test_window_batched_unbounded_no_part(b_gen, batch_size): @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches @pytest.mark.parametrize('b_gen', all_basic_gens + [decimal_gen_32bit, decimal_gen_128bit], ids=meta_idfn('data:')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_window_batched_unbounded(b_gen, batch_size): conf = {'spark.rapids.sql.batchSizeBytes': batch_size, 'spark.rapids.sql.castFloatToDecimal.enabled': True} @@ -490,7 +488,6 @@ def test_window_batched_unbounded(b_gen, batch_size): # the order returned should be consistent because the data ends up in a single task (no partitioning) @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches @pytest.mark.parametrize('b_gen', all_basic_gens + [decimal_gen_32bit, decimal_gen_128bit], ids=meta_idfn('data:')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_rows_based_running_window_unpartitioned(b_gen, batch_size): conf = {'spark.rapids.sql.batchSizeBytes': batch_size, 'spark.rapids.sql.castFloatToDecimal.enabled': True} @@ -1686,7 +1683,6 @@ def test_window_first_last_nth(data_gen): @pytest.mark.skipif(is_before_spark_320(), reason='IGNORE NULLS clause is not supported for FIRST(), LAST() and NTH_VALUE in Spark 3.1.x') @pytest.mark.parametrize('data_gen', all_basic_gens_no_null + decimal_gens + _nested_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_window_first_last_nth_ignore_nulls(data_gen): assert_gpu_and_cpu_are_equal_sql( # Coalesce is to make sure that first and last, which are non-deterministic become deterministic From eb32703853c04f0e0534729de0b89789a65c7a20 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Mon, 27 Nov 2023 21:20:44 +0800 Subject: [PATCH 14/46] Fix inmatch cases --- integration_tests/src/main/python/aqe_test.py | 1 + .../src/main/python/cast_test.py | 3 ++ integration_tests/src/main/python/csv_test.py | 1 + .../src/main/python/date_time_test.py | 1 + .../src/main/python/mortgage_test.py | 1 - .../com/nvidia/spark/rapids/GpuCSVScan.scala | 3 +- .../com/nvidia/spark/rapids/RapidsMeta.scala | 44 ++++++++++++++----- .../catalyst/json/rapids/GpuJsonScan.scala | 3 +- 8 files changed, 42 insertions(+), 15 deletions(-) diff --git a/integration_tests/src/main/python/aqe_test.py b/integration_tests/src/main/python/aqe_test.py index e80e1a2db2b..189bef329d7 100755 --- a/integration_tests/src/main/python/aqe_test.py +++ b/integration_tests/src/main/python/aqe_test.py @@ -196,6 +196,7 @@ def do_it(spark): @ignore_order(local=True) @allow_non_gpu('BroadcastNestedLoopJoinExec', 'Cast', 'DateSub', *db_113_cpu_bnlj_join_allow) @pytest.mark.parametrize('join', joins, ids=idfn) +@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_aqe_join_reused_exchange_inequality_condition(spark_tmp_path, join): data_path = spark_tmp_path + '/PARQUET_DATA' def prep(spark): diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index 60b718ba15f..4d43cd3322e 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -91,6 +91,7 @@ def test_cast_string_date_valid_format(): # Spark 320+ and databricks support Ansi mode when casting string to date # This means an exception will be thrown when casting invalid string to date on Spark 320+ or databricks # test Spark versions < 3.2.0 and non databricks, ANSI mode +@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') @pytest.mark.skipif(not is_before_spark_320(), reason="ansi cast(string as date) throws exception only in 3.2.0+ or db") def test_cast_string_date_invalid_ansi_before_320(): data_rows = [(v,) for v in values_string_to_data] @@ -323,6 +324,7 @@ def test_cast_array_with_unmatched_element_to_string(data_gen, legacy): @pytest.mark.parametrize('data_gen', basic_map_gens_for_cast_to_string, ids=idfn) @pytest.mark.parametrize('legacy', ['true', 'false']) +@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_cast_map_to_string(data_gen, legacy): _assert_cast_to_string_equal( data_gen, @@ -342,6 +344,7 @@ def test_cast_map_with_unmatched_element_to_string(data_gen, legacy): @pytest.mark.parametrize('data_gen', [StructGen([[str(i), gen] for i, gen in enumerate(basic_array_struct_gens_for_cast_to_string)] + [["map", MapGen(ByteGen(nullable=False), null_gen)]])], ids=idfn) @pytest.mark.parametrize('legacy', ['true', 'false']) +@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_cast_struct_to_string(data_gen, legacy): _assert_cast_to_string_equal( data_gen, diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index 02c12e333e0..c10221a4407 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -248,6 +248,7 @@ def read_impl(spark): @pytest.mark.parametrize('read_func', [read_csv_df, read_csv_sql]) @pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) +@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_basic_csv_read(std_input_path, name, schema, options, read_func, v1_enabled_list, ansi_enabled, spark_tmp_table_factory): updated_conf=copy_and_update(_enable_all_types_conf, { 'spark.sql.sources.useV1SourceList': v1_enabled_list, diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index a731952ed2c..8ce8d5c5e00 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -64,6 +64,7 @@ def test_interval_seconds_overflow_exception(): error_message="IllegalArgumentException") @pytest.mark.parametrize('data_gen', vals, ids=idfn) +@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_timeadd_from_subquery(data_gen): def fun(spark): diff --git a/integration_tests/src/main/python/mortgage_test.py b/integration_tests/src/main/python/mortgage_test.py index a6b283e7da8..aed9aa63c85 100644 --- a/integration_tests/src/main/python/mortgage_test.py +++ b/integration_tests/src/main/python/mortgage_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_iterator -from conftest import is_not_utc from marks import approximate_float, incompat, ignore_order, allow_non_gpu, limit @incompat diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala index 7f078cb4db2..84fd024b935 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala @@ -168,7 +168,8 @@ object GpuCSVScan { GpuCsvUtils.dateFormatInRead(parsedOptions), parseString = true) } - if (types.contains(TimestampType)) { + // Date type needs to be checked even it has no time zone info as its fields. + if (types.contains(TimestampType) || types.contains(DateType)) { meta.checkTimeZoneId(parsedOptions.zoneId) GpuTextBasedDateUtils.tagCudfFormat(meta, GpuCsvUtils.timestampFormatInRead(parsedOptions), parseString = true) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 4d2e8605c52..17e02a27cdd 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.python.AggregateInPandasExec import org.apache.spark.sql.rapids.TimeZoneDB import org.apache.spark.sql.rapids.aggregate.{CpuToGpuAggregateBufferConverter, GpuToCpuAggregateBufferConverter} import org.apache.spark.sql.rapids.execution.{GpuBroadcastHashJoinMetaBase, GpuBroadcastNestedLoopJoinMetaBase} -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StringType, StructType} trait DataFromReplacementRule { val operationName: String @@ -1085,22 +1085,22 @@ abstract class BaseExprMeta[INPUT <: Expression]( // There are 4 levels of timezone check in GPU plan tag phase: // Level 1: Check whether an expression is related to timezone. This is achieved by - // [[needTimezoneCheck]] below. + // [[needTimeZoneCheck]] below. // Level 2: Check on golden configuration 'spark.rapids.sql.nonUtc.enabled'. If // yes, we pass to next level timezone check. If not, we only pass UTC case as before. // Level 3: Check related expression has been implemented with timezone. There is a - // toggle flag [[isTimezoneSupported]] for this. If false, fallback to UTC-only check as + // toggle flag [[isTimeZoneSupported]] for this. If false, fallback to UTC-only check as // before. If yes, move to next level check. When we add timezone support for a related - // function. [[isTimezoneSupported]] should be override as true. + // function. [[isTimeZoneSupported]] should be override as true. // Level 4: Check whether the desired timezone is supported by Gpu kernel. def checkExprForTimezone(): Unit = { // Level 1 check - if (!needTimezoneCheck) return + if (!needTimeZoneCheck) return // TODO: Level 2 check // Level 3 check - if (!isTimezoneSupported) return checkUTCTimezone(this) + if (!isTimeZoneSupported) return checkUTCTimezone(this) // Level 4 check if (TimeZoneDB.isSupportedTimezone(getZoneId())) { @@ -1122,31 +1122,51 @@ abstract class BaseExprMeta[INPUT <: Expression]( } // Level 1 timezone checking flag - // Both [[isTimezoneSupported]] and [[needTimezoneCheck]] are needed to check whether timezone + // Both [[isTimeZoneSupported]] and [[needTimeZoneCheck]] are needed to check whether timezone // check needed. For cast expression, only some cases are needed pending on its data type and // its child's data type. // //+------------------------+-------------------+-----------------------------------------+ - //| Value | needTimezoneCheck | isTimezoneSupported | + //| Value | needTimeZoneCheck | isTimeZoneSupported | //+------------------------+-------------------+-----------------------------------------+ //| TimezoneAwareExpression| True | False by default, True when implemented | //| UTCTimestamp | True | False by default, True when implemented | //| Others | False | N/A (will not be checked) | //+------------------------+-------------------+-----------------------------------------+ - lazy val needTimezoneCheck: Boolean = { + lazy val needTimeZoneCheck: Boolean = { wrapped match { case _: TimeZoneAwareExpression => // CurrentDate expression will not go through this even it's a `TimeZoneAwareExpression`. // It will be treated as literal in Rapids. - if (wrapped.isInstanceOf[Cast]) wrapped.asInstanceOf[Cast].needsTimeZone else true + if (wrapped.isInstanceOf[Cast]) { + val cast = wrapped.asInstanceOf[Cast] + needsTimeZone(cast.child.dataType, cast.dataType) + } else { + true + } case _: UTCTimestamp => true case _ => false } } + // This is to workaround Spark's issue where `needsTimezone``` doesn't consider complex types to + // string which is timezone related. (incl. struct/map/list to string). + // This is used for complex types to string. + private[this] def needsTimeZone(from: DataType, to: DataType): Boolean = (from, to) match { + case (ArrayType(fromType, _), StringType) => needsTimeZone(fromType, to) + case (MapType(fromKey, fromValue, _), StringType) => + needsTimeZone(fromKey, to) || needsTimeZone(fromValue, to) + case (StructType(fromFields), StringType) => + fromFields.exists { + case fromField => + needsTimeZone(fromField.dataType, to) + } + case _ => Cast.needsTimeZone(from, to) + } + // Level 3 timezone checking flag, need to override to true when supports timezone in functions - // Useless if it's not timezone related expression defined in [[needTimezoneCheck]] - val isTimezoneSupported: Boolean = false + // Useless if it's not timezone related expression defined in [[needTimeZoneCheck]] + val isTimeZoneSupported: Boolean = false /** * Timezone check which only allows UTC timezone. This is consistent with previous behavior. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala index c4840839616..81eec957ea8 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala @@ -168,7 +168,8 @@ object GpuJsonScan { GpuJsonUtils.dateFormatInRead(parsedOptions), parseString = true) } - if (types.contains(TimestampType)) { + // Date type needs to be checked even it has no time zone info as its fields. + if (types.contains(TimestampType) || types.contains(DateType)) { meta.checkTimeZoneId(parsedOptions.zoneId) GpuTextBasedDateUtils.tagCudfFormat(meta, GpuJsonUtils.timestampFormatInRead(parsedOptions), parseString = true) From f0f6164560fcf488e674ca703cd15ee103256bf9 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Mon, 27 Nov 2023 21:30:39 +0800 Subject: [PATCH 15/46] Revert before commit --- integration_tests/run_pyspark_from_build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index f6e32c72161..4d5770d4b57 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -224,7 +224,7 @@ else fi # time zone will be tested; use export TZ=time_zone_name before run this script - TZ=${TZ:-UTC} + TZ="Asia/Shanghai" # Set the Delta log cache size to prevent the driver from caching every Delta log indefinitely export PYSP_TEST_spark_driver_extraJavaOptions="-ea -Duser.timezone=$TZ -Ddelta.log.cacheSize=10 $COVERAGE_SUBMIT_FLAGS" From e554f4e3cd75cd9ce3067c593b80f51d8e7af9e6 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Mon, 27 Nov 2023 22:10:26 +0800 Subject: [PATCH 16/46] Fix --- integration_tests/src/main/python/cast_test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index 4d43cd3322e..725b95a6590 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -112,6 +112,7 @@ def test_cast_string_date_valid_ansi(): # test Spark versions >= 320, ANSI mode @pytest.mark.skipif(is_before_spark_320(), reason="ansi cast(string as date) throws exception only in 3.2.0+") +@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') @pytest.mark.parametrize('invalid', invalid_values_string_to_date) def test_cast_string_date_invalid_ansi(invalid): assert_gpu_and_cpu_error( From d9fe75231b2545f569a83e2b4d4a1a481945f1df Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Tue, 28 Nov 2023 17:07:48 +0800 Subject: [PATCH 17/46] Fix --- .../src/main/python/cast_test.py | 5 ---- .../src/main/python/date_time_test.py | 2 -- .../nvidia/spark/rapids/GpuOverrides.scala | 3 +++ .../com/nvidia/spark/rapids/RapidsMeta.scala | 26 +++++++++++++++---- .../spark/sql/rapids/GpuOrcFileFormat.scala | 12 +++++++++ 5 files changed, 36 insertions(+), 12 deletions(-) diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index 725b95a6590..dbb41b60bb7 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -61,7 +61,6 @@ def test_cast_nested(data_gen, to_type): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).select(f.col('a').cast(to_type))) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') @datagen_overrides(seed=0, reason="https://github.com/NVIDIA/spark-rapids/issues/9781") def test_cast_string_date_valid_format(): # In Spark 3.2.0+ the valid format changed, and we cannot support all of the format. @@ -91,7 +90,6 @@ def test_cast_string_date_valid_format(): # Spark 320+ and databricks support Ansi mode when casting string to date # This means an exception will be thrown when casting invalid string to date on Spark 320+ or databricks # test Spark versions < 3.2.0 and non databricks, ANSI mode -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') @pytest.mark.skipif(not is_before_spark_320(), reason="ansi cast(string as date) throws exception only in 3.2.0+ or db") def test_cast_string_date_invalid_ansi_before_320(): data_rows = [(v,) for v in values_string_to_data] @@ -101,7 +99,6 @@ def test_cast_string_date_invalid_ansi_before_320(): 'spark.sql.ansi.enabled': 'true'}, ) # test Spark versions >= 320 and databricks, ANSI mode, valid values -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') @pytest.mark.skipif(is_before_spark_320(), reason="Spark versions(< 320) not support Ansi mode when casting string to date") def test_cast_string_date_valid_ansi(): data_rows = [(v,) for v in valid_values_string_to_date] @@ -112,7 +109,6 @@ def test_cast_string_date_valid_ansi(): # test Spark versions >= 320, ANSI mode @pytest.mark.skipif(is_before_spark_320(), reason="ansi cast(string as date) throws exception only in 3.2.0+") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') @pytest.mark.parametrize('invalid', invalid_values_string_to_date) def test_cast_string_date_invalid_ansi(invalid): assert_gpu_and_cpu_error( @@ -145,7 +141,6 @@ def test_try_cast_fallback_340(invalid): 'spark.sql.ansi.enabled': True}) # test all Spark versions, non ANSI mode, invalid value will be converted to NULL -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_cast_string_date_non_ansi(): data_rows = [(v,) for v in values_string_to_data] assert_gpu_and_cpu_are_equal_collect( diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index 8ce8d5c5e00..1d984193f9e 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -25,7 +25,6 @@ # We only support literal intervals for TimeSub vals = [(-584, 1563), (1943, 1101), (2693, 2167), (2729, 0), (44, 1534), (2635, 3319), (1885, -2828), (0, 2463), (932, 2286), (0, 0)] - @pytest.mark.parametrize('data_gen', vals, ids=idfn) @pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_timesub(data_gen): @@ -459,7 +458,6 @@ def test_gettimestamp_ansi_exception(): supported_date_formats = ['yyyy-MM-dd', 'yyyy-MM', 'yyyy/MM/dd', 'yyyy/MM', 'dd/MM/yyyy', 'MM-dd', 'MM/dd', 'dd-MM', 'dd/MM'] - @pytest.mark.parametrize('date_format', supported_date_formats, ids=idfn) @pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) @pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 8c4a90b7195..145985980c2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -694,6 +694,9 @@ object GpuOverrides extends Logging { def isOrContainsFloatingPoint(dataType: DataType): Boolean = TrampolineUtil.dataTypeExistsRecursively(dataType, dt => dt == FloatType || dt == DoubleType) + def isContainsDateOrTimestamp(dataType: DataType): Boolean = + TrampolineUtil.dataTypeExistsRecursively(dataType, dt => dt == TimestampType || dt == DateType) + /** Tries to predict whether an adaptive plan will end up with data on the GPU or not. */ def probablyGpuPlan(adaptivePlan: AdaptiveSparkPlanExec, conf: RapidsConf): Boolean = { def findRootProcessingNode(plan: SparkPlan): SparkPlan = plan match { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 17e02a27cdd..979076f478a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.python.AggregateInPandasExec import org.apache.spark.sql.rapids.TimeZoneDB import org.apache.spark.sql.rapids.aggregate.{CpuToGpuAggregateBufferConverter, GpuToCpuAggregateBufferConverter} import org.apache.spark.sql.rapids.execution.{GpuBroadcastHashJoinMetaBase, GpuBroadcastNestedLoopJoinMetaBase} -import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StringType, StructType} +import org.apache.spark.sql.types.{ArrayType, DataType, DateType, MapType, StringType, StructType, TimestampNTZType, TimestampType} trait DataFromReplacementRule { val operationName: String @@ -1149,10 +1149,17 @@ abstract class BaseExprMeta[INPUT <: Expression]( } } - // This is to workaround Spark's issue where `needsTimezone``` doesn't consider complex types to - // string which is timezone related. (incl. struct/map/list to string). - // This is used for complex types to string. + // Mostly base on Spark existing [[Cast.needsTimeZone]] method. Two changes are made: + // 1. Backport commit https://github.com/apache/spark/pull/40524 merged since Spark 3.5 + // 2. Existing `needsTimezone``` doesn't consider complex types to string which is timezone + // related. (incl. struct/map/list to string). private[this] def needsTimeZone(from: DataType, to: DataType): Boolean = (from, to) match { + case (StringType, TimestampType) => true + case (TimestampType, StringType) => true + case (DateType, TimestampType) => true + case (TimestampType, DateType) => true + case (TimestampType, TimestampNTZType) => true + case (TimestampNTZType, TimestampType) => true case (ArrayType(fromType, _), StringType) => needsTimeZone(fromType, to) case (MapType(fromKey, fromValue, _), StringType) => needsTimeZone(fromKey, to) || needsTimeZone(fromValue, to) @@ -1161,7 +1168,16 @@ abstract class BaseExprMeta[INPUT <: Expression]( case fromField => needsTimeZone(fromField.dataType, to) } - case _ => Cast.needsTimeZone(from, to) + case (ArrayType(fromType, _), ArrayType(toType, _)) => needsTimeZone(fromType, toType) + case (MapType(fromKey, fromValue, _), MapType(toKey, toValue, _)) => + needsTimeZone(fromKey, toKey) || needsTimeZone(fromValue, toValue) + case (StructType(fromFields), StructType(toFields)) => + fromFields.length == toFields.length && + fromFields.zip(toFields).exists { + case (fromField, toField) => + needsTimeZone(fromField.dataType, toField.dataType) + } + case _ => false } // Level 3 timezone checking flag, need to override to true when supports timezone in functions diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala index 9e50a993537..c696b25fe6f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala @@ -16,6 +16,8 @@ package org.apache.spark.sql.rapids +import java.time.ZoneId + import ai.rapids.cudf._ import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.shims.OrcShims @@ -30,6 +32,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.orc.{OrcFileFormat, OrcOptions, OrcUtils} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ object GpuOrcFileFormat extends Logging { @@ -75,6 +78,15 @@ object GpuOrcFileFormat extends Logging { "If bloom filter is not required, unset \"orc.bloom.filter.columns\"") } + val types = schema.map(_.dataType).toSet + if (types.exists(GpuOverrides.isContainsDateOrTimestamp(_))) { + if (!TimeZoneDB.isUTCTimezone()) { + meta.willNotWorkOnGpu("Only UTC timezone is supported for ORC. " + + s"Current timezone settings: (JVM : ${ZoneId.systemDefault()}, " + + s"session: ${SQLConf.get.sessionLocalTimeZone}). ") + } + } + FileFormatChecks.tag(meta, schema, OrcFormatType, WriteFileOp) val sqlConf = spark.sessionState.conf From 78e58048d316430d1e6af5d0d8ad821b2dfdd6af Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Tue, 28 Nov 2023 18:45:32 +0800 Subject: [PATCH 18/46] Comments --- .../scala/com/nvidia/spark/rapids/GpuCSVScan.scala | 11 ++++++++++- .../spark/sql/catalyst/json/rapids/GpuJsonScan.scala | 10 +++++++++- .../apache/spark/sql/rapids/GpuOrcFileFormat.scala | 4 ++++ 3 files changed, 23 insertions(+), 2 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala index 84fd024b935..cc25b4a50ef 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala @@ -168,7 +168,16 @@ object GpuCSVScan { GpuCsvUtils.dateFormatInRead(parsedOptions), parseString = true) } - // Date type needs to be checked even it has no time zone info as its fields. + // For date type, timezone needs to be checked also. This is because JVM timezone is used + // to get days offset before rebasing Julian to Gregorian in Spark while not in Rapids. + // + // In details, for CSV data format, Spark uses dateFormatter to parse string as date data + // type which utilizes [[org.apache.spark.sql.catalyst.DateFormatter]]. And CSV format + // (e.g., [[UnivocityParser]]), it uses [[LegacyFastDateFormatter]] which is based on + // Apache Commons FastDateFormat. It parse string into Java util.Date base on JVM default + // timezone. From Java util.Date, it's converted into java.sql.Date type. + // By leveraging [[JavaDateTimeUtils]], it finally do `rebaseJulianToGregorianDays` considering + // its offset to UTC timezone. if (types.contains(TimestampType) || types.contains(DateType)) { meta.checkTimeZoneId(parsedOptions.zoneId) GpuTextBasedDateUtils.tagCudfFormat(meta, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala index 81eec957ea8..71511e2434a 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala @@ -168,7 +168,15 @@ object GpuJsonScan { GpuJsonUtils.dateFormatInRead(parsedOptions), parseString = true) } - // Date type needs to be checked even it has no time zone info as its fields. + // For date type, timezone needs to be checked also. This is because JVM timezone is used + // to get days offset before rebasing Julian to Gregorian in Spark while not in Rapids. + // + // In details, for Json data format, Spark uses dateFormatter to parse string as date data + // type which utilizes [[org.apache.spark.sql.catalyst.DateFormatter]]. For Json format, it uses + // [[LegacyFastDateFormatter]] which is based on Apache Commons FastDateFormat. It parse string + // into Java util.Date base on JVM default timezone. From Java util.Date, it's converted into + // java.sql.Date type. By leveraging [[JavaDateTimeUtils]], it finally do + // `rebaseJulianToGregorianDays` considering its offset to UTC timezone. if (types.contains(TimestampType) || types.contains(DateType)) { meta.checkTimeZoneId(parsedOptions.zoneId) GpuTextBasedDateUtils.tagCudfFormat(meta, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala index c696b25fe6f..737c74ca619 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala @@ -78,6 +78,10 @@ object GpuOrcFileFormat extends Logging { "If bloom filter is not required, unset \"orc.bloom.filter.columns\"") } + // For date type, timezone needs to be checked also. This is because JVM timezone and UTC + // timezone offset is considered when getting [[java.sql.date]] from + // [[org.apache.spark.sql.execution.datasources.DaysWritable]] object + // which is a subclass of [[org.apache.hadoop.hive.serde2.io.DateWritable]]. val types = schema.map(_.dataType).toSet if (types.exists(GpuOverrides.isContainsDateOrTimestamp(_))) { if (!TimeZoneDB.isUTCTimezone()) { From 2ad72c8dca5193022b6524927fe473de2233ae38 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Tue, 28 Nov 2023 19:12:07 +0800 Subject: [PATCH 19/46] Fix --- .../com/nvidia/spark/rapids/RapidsMeta.scala | 27 ++++++------------- 1 file changed, 8 insertions(+), 19 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 979076f478a..adc4e97abc2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.python.AggregateInPandasExec import org.apache.spark.sql.rapids.TimeZoneDB import org.apache.spark.sql.rapids.aggregate.{CpuToGpuAggregateBufferConverter, GpuToCpuAggregateBufferConverter} import org.apache.spark.sql.rapids.execution.{GpuBroadcastHashJoinMetaBase, GpuBroadcastNestedLoopJoinMetaBase} -import org.apache.spark.sql.types.{ArrayType, DataType, DateType, MapType, StringType, StructType, TimestampNTZType, TimestampType} +import org.apache.spark.sql.types.{ArrayType, DataType, DateType, MapType, StringType, StructType} trait DataFromReplacementRule { val operationName: String @@ -1150,16 +1150,12 @@ abstract class BaseExprMeta[INPUT <: Expression]( } // Mostly base on Spark existing [[Cast.needsTimeZone]] method. Two changes are made: - // 1. Backport commit https://github.com/apache/spark/pull/40524 merged since Spark 3.5 - // 2. Existing `needsTimezone``` doesn't consider complex types to string which is timezone + // 1. Override date related based on https://github.com/apache/spark/pull/40524 merged + // 2. Existing `needsTimezone` doesn't consider complex types to string which is timezone // related. (incl. struct/map/list to string). private[this] def needsTimeZone(from: DataType, to: DataType): Boolean = (from, to) match { - case (StringType, TimestampType) => true - case (TimestampType, StringType) => true - case (DateType, TimestampType) => true - case (TimestampType, DateType) => true - case (TimestampType, TimestampNTZType) => true - case (TimestampNTZType, TimestampType) => true + case (StringType, DateType) => false + case (DateType, StringType) => false case (ArrayType(fromType, _), StringType) => needsTimeZone(fromType, to) case (MapType(fromKey, fromValue, _), StringType) => needsTimeZone(fromKey, to) || needsTimeZone(fromValue, to) @@ -1168,16 +1164,9 @@ abstract class BaseExprMeta[INPUT <: Expression]( case fromField => needsTimeZone(fromField.dataType, to) } - case (ArrayType(fromType, _), ArrayType(toType, _)) => needsTimeZone(fromType, toType) - case (MapType(fromKey, fromValue, _), MapType(toKey, toValue, _)) => - needsTimeZone(fromKey, toKey) || needsTimeZone(fromValue, toValue) - case (StructType(fromFields), StructType(toFields)) => - fromFields.length == toFields.length && - fromFields.zip(toFields).exists { - case (fromField, toField) => - needsTimeZone(fromField.dataType, toField.dataType) - } - case _ => false + // Avoid copying full implementation here. Otherwise needs to create shim for TimestampNTZ + // since Spark 3.4.0 + case _ => Cast.needsTimeZone(from, to) } // Level 3 timezone checking flag, need to override to true when supports timezone in functions From 7c734c0441e436ffe945a8ab048fa5c871314d67 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 00:52:49 +0800 Subject: [PATCH 20/46] Fix failed cases --- integration_tests/run_pyspark_from_build.sh | 2 +- .../src/main/python/json_test.py | 4 ++- .../src/main/python/parquet_test.py | 8 ++--- .../com/nvidia/spark/rapids/GpuCSVScan.scala | 29 ++++++++++--------- .../nvidia/spark/rapids/GpuOverrides.scala | 5 +++- .../com/nvidia/spark/rapids/RapidsMeta.scala | 4 +-- .../catalyst/json/rapids/GpuJsonScan.scala | 28 +++++++++++------- .../sql/hive/rapids/HiveProviderImpl.scala | 12 ++++++++ .../spark/sql/rapids/GpuOrcFileFormat.scala | 2 +- 9 files changed, 61 insertions(+), 33 deletions(-) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index 4d5770d4b57..f6e32c72161 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -224,7 +224,7 @@ else fi # time zone will be tested; use export TZ=time_zone_name before run this script - TZ="Asia/Shanghai" + TZ=${TZ:-UTC} # Set the Delta log cache size to prevent the driver from caching every Delta log indefinitely export PYSP_TEST_spark_driver_extraJavaOptions="-ea -Duser.timezone=$TZ -Ddelta.log.cacheSize=10 $COVERAGE_SUBMIT_FLAGS" diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 41571a203d5..16e28b724c2 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -181,10 +181,12 @@ def test_json_date_formats_round_trip(spark_tmp_path, date_format, v1_enabled_li "'T'HH:mm[:ss]", "'T'HH:mm"] +not_utc_allow=['BatchScanExec'] if is_not_utc() else [] + @pytest.mark.parametrize('ts_part', json_supported_ts_parts) @pytest.mark.parametrize('date_format', json_supported_date_formats) @pytest.mark.parametrize('v1_enabled_list', ["", "json"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*not_utc_allow) def test_json_ts_formats_round_trip(spark_tmp_path, date_format, ts_part, v1_enabled_list): full_format = date_format + ts_part data_gen = TimestampGen() diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py index f6cc2a0141b..dd29be6bd3b 100644 --- a/integration_tests/src/main/python/parquet_test.py +++ b/integration_tests/src/main/python/parquet_test.py @@ -313,13 +313,13 @@ def test_parquet_pred_push_round_trip(spark_tmp_path, parquet_gen, read_func, v1 lambda spark: rf(spark).select(f.col('a') >= s0), conf=all_confs) +@pytest.mark.skipif(is_not_utc(), reason="LEGACY datetime rebase mode is only supported for UTC timezone") @pytest.mark.parametrize('parquet_gens', [parquet_nested_datetime_gen], ids=idfn) @pytest.mark.parametrize('ts_type', parquet_ts_write_options) @pytest.mark.parametrize('ts_rebase_write', [('CORRECTED', 'LEGACY'), ('LEGACY', 'CORRECTED')]) @pytest.mark.parametrize('ts_rebase_read', [('CORRECTED', 'LEGACY'), ('LEGACY', 'CORRECTED')]) @pytest.mark.parametrize('reader_confs', reader_opt_confs) @pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_parquet_read_roundtrip_datetime_with_legacy_rebase(spark_tmp_path, parquet_gens, ts_type, ts_rebase_write, ts_rebase_read, reader_confs, v1_enabled_list): @@ -356,10 +356,10 @@ def test_parquet_decimal_read_legacy(spark_tmp_path, parquet_gens, read_func, re all_confs = copy_and_update(reader_confs, {'spark.sql.sources.useV1SourceList': v1_enabled_list}) assert_gpu_and_cpu_are_equal_collect(read_func(data_path), conf=all_confs) +@pytest.mark.skipif(is_not_utc(), reason="LEGACY datetime rebase mode is only supported for UTC timezone") @pytest.mark.parametrize('reader_confs', reader_opt_confs) @pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) @pytest.mark.parametrize('batch_size', [100, INT_MAX]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_parquet_simple_partitioned_read(spark_tmp_path, v1_enabled_list, reader_confs, batch_size): # Once https://github.com/NVIDIA/spark-rapids/issues/133 and https://github.com/NVIDIA/spark-rapids/issues/132 are fixed # we should go with a more standard set of generators @@ -389,9 +389,9 @@ def test_parquet_simple_partitioned_read(spark_tmp_path, v1_enabled_list, reader # In this we are reading the data, but only reading the key the data was partitioned by +@pytest.mark.skipif(is_not_utc(), reason="LEGACY datetime rebase mode is only supported for UTC timezone") @pytest.mark.parametrize('reader_confs', reader_opt_confs) @pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_parquet_partitioned_read_just_partitions(spark_tmp_path, v1_enabled_list, reader_confs): parquet_gens = [byte_gen] gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] @@ -534,7 +534,7 @@ def read_and_remove(spark): @pytest.mark.parametrize('reader_confs', reader_opt_confs) @pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@pytest.mark.skipif(is_not_utc(), reason="LEGACY datetime rebase mode is only supported for UTC timezone") def test_parquet_read_merge_schema(spark_tmp_path, v1_enabled_list, reader_confs): # Once https://github.com/NVIDIA/spark-rapids/issues/133 and https://github.com/NVIDIA/spark-rapids/issues/132 are fixed # we should go with a more standard set of generators diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala index cc25b4a50ef..97138843065 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.LegacyTimeParserPolicy +import org.apache.spark.sql.rapids.{LegacyTimeParserPolicy, TimeZoneDB} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.vectorized.ColumnarBatch @@ -166,20 +166,23 @@ object GpuCSVScan { if (types.contains(DateType)) { GpuTextBasedDateUtils.tagCudfFormat(meta, GpuCsvUtils.dateFormatInRead(parsedOptions), parseString = true) + + // For date type, timezone needs to be checked also. This is because JVM timezone is used + // to get days offset before rebasing Julian to Gregorian in Spark while not in Rapids. + // + // In details, for CSV data format, Spark uses dateFormatter to parse string as date data + // type which utilizes [[org.apache.spark.sql.catalyst.DateFormatter]]. And CSV format + // (e.g., [[UnivocityParser]]), it uses [[LegacyFastDateFormatter]] which is based on + // Apache Commons FastDateFormat. It parse string into Java util.Date base on JVM default + // timezone. From Java util.Date, it's converted into java.sql.Date type. + // By leveraging [[JavaDateTimeUtils]], it finally do `rebaseJulianToGregorianDays` + // considering its offset to UTC timezone. + if (!TimeZoneDB.isUTCTimezone(parsedOptions.zoneId)) { + meta.willNotWorkOnGpu(s"Not supported timezone type ${parsedOptions.zoneId}.") + } } - // For date type, timezone needs to be checked also. This is because JVM timezone is used - // to get days offset before rebasing Julian to Gregorian in Spark while not in Rapids. - // - // In details, for CSV data format, Spark uses dateFormatter to parse string as date data - // type which utilizes [[org.apache.spark.sql.catalyst.DateFormatter]]. And CSV format - // (e.g., [[UnivocityParser]]), it uses [[LegacyFastDateFormatter]] which is based on - // Apache Commons FastDateFormat. It parse string into Java util.Date base on JVM default - // timezone. From Java util.Date, it's converted into java.sql.Date type. - // By leveraging [[JavaDateTimeUtils]], it finally do `rebaseJulianToGregorianDays` considering - // its offset to UTC timezone. - if (types.contains(TimestampType) || types.contains(DateType)) { - meta.checkTimeZoneId(parsedOptions.zoneId) + if (types.contains(TimestampType)) { GpuTextBasedDateUtils.tagCudfFormat(meta, GpuCsvUtils.timestampFormatInRead(parsedOptions), parseString = true) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 145985980c2..f27d97d48c6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -694,9 +694,12 @@ object GpuOverrides extends Logging { def isOrContainsFloatingPoint(dataType: DataType): Boolean = TrampolineUtil.dataTypeExistsRecursively(dataType, dt => dt == FloatType || dt == DoubleType) - def isContainsDateOrTimestamp(dataType: DataType): Boolean = + def isOrContainsDateOrTimestamp(dataType: DataType): Boolean = TrampolineUtil.dataTypeExistsRecursively(dataType, dt => dt == TimestampType || dt == DateType) + def isOrContainsTimestamp(dataType: DataType): Boolean = + TrampolineUtil.dataTypeExistsRecursively(dataType, dt => dt == TimestampType) + /** Tries to predict whether an adaptive plan will end up with data on the GPU or not. */ def probablyGpuPlan(adaptivePlan: AdaptiveSparkPlanExec, conf: RapidsConf): Boolean = { def findRootProcessingNode(plan: SparkPlan): SparkPlan = plan match { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index adc4e97abc2..82167d30fe1 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -384,13 +384,13 @@ abstract class RapidsMeta[INPUT <: BASE, BASE, OUTPUT <: BASE]( def checkTimeZoneId(sessionZoneId: ZoneId): Unit = { // Both of the Spark session time zone and JVM's default time zone should be UTC. if (!TimeZoneDB.isSupportedTimezone(sessionZoneId)) { - willNotWorkOnGpu("Only UTC zone id is supported. " + + willNotWorkOnGpu("Not supported zone id. " + s"Actual session local zone id: $sessionZoneId") } val defaultZoneId = ZoneId.systemDefault() if (!TimeZoneDB.isSupportedTimezone(defaultZoneId)) { - willNotWorkOnGpu(s"Only UTC zone id is supported. Actual default zone id: $defaultZoneId") + willNotWorkOnGpu(s"Not supported zone id. Actual default zone id: $defaultZoneId") } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala index 71511e2434a..207a7c2cf3f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.execution.datasources.{PartitionedFile, Partitioning import org.apache.spark.sql.execution.datasources.v2.{FileScan, TextBasedFileScan} import org.apache.spark.sql.execution.datasources.v2.json.JsonScan import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids.TimeZoneDB import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types.{DataType, DateType, DecimalType, DoubleType, FloatType, StringType, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -166,19 +167,26 @@ object GpuJsonScan { if (types.contains(DateType)) { GpuTextBasedDateUtils.tagCudfFormat(meta, GpuJsonUtils.dateFormatInRead(parsedOptions), parseString = true) + + // For date type, timezone needs to be checked also. This is because JVM timezone is used + // to get days offset before rebasing Julian to Gregorian in Spark while not in Rapids. + // + // In details, for Json data format, Spark uses dateFormatter to parse string as date data + // type which utilizes [[org.apache.spark.sql.catalyst.DateFormatter]]. For Json format, it + // uses [[LegacyFastDateFormatter]] which is based on Apache Commons FastDateFormat. It parse + // string into Java util.Date base on JVM default timezone. From Java util.Date, it's + // converted into java.sql.Date type. By leveraging [[JavaDateTimeUtils]], it finally do + // `rebaseJulianToGregorianDays` considering its offset to UTC timezone. + if(!TimeZoneDB.isUTCTimezone(parsedOptions.zoneId)){ + meta.willNotWorkOnGpu(s"Not supported timezone type ${parsedOptions.zoneId}.") + } } - // For date type, timezone needs to be checked also. This is because JVM timezone is used - // to get days offset before rebasing Julian to Gregorian in Spark while not in Rapids. - // - // In details, for Json data format, Spark uses dateFormatter to parse string as date data - // type which utilizes [[org.apache.spark.sql.catalyst.DateFormatter]]. For Json format, it uses - // [[LegacyFastDateFormatter]] which is based on Apache Commons FastDateFormat. It parse string - // into Java util.Date base on JVM default timezone. From Java util.Date, it's converted into - // java.sql.Date type. By leveraging [[JavaDateTimeUtils]], it finally do - // `rebaseJulianToGregorianDays` considering its offset to UTC timezone. if (types.contains(TimestampType) || types.contains(DateType)) { - meta.checkTimeZoneId(parsedOptions.zoneId) + if (!TimeZoneDB.isUTCTimezone(parsedOptions.zoneId)) { + meta.willNotWorkOnGpu(s"Not supported timezone type ${parsedOptions.zoneId}.") + } + GpuTextBasedDateUtils.tagCudfFormat(meta, GpuJsonUtils.timestampFormatInRead(parsedOptions), parseString = true) } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala index 476c3f75f3b..99d33901e22 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hive.rapids import java.nio.charset.Charset +import java.time.ZoneId import com.google.common.base.Charsets import com.nvidia.spark.RapidsUDF @@ -31,6 +32,8 @@ import org.apache.spark.sql.hive.{HiveGenericUDF, HiveSimpleUDF} import org.apache.spark.sql.hive.execution.HiveTableScanExec import org.apache.spark.sql.hive.rapids.GpuHiveTextFileUtils._ import org.apache.spark.sql.hive.rapids.shims.HiveProviderCmdShims +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids.TimeZoneDB import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types._ @@ -289,6 +292,15 @@ class HiveProviderImpl extends HiveProviderCmdShims { willNotWorkOnGpu("reading of decimal typed values has been disabled set " + s"${RapidsConf.ENABLE_READ_HIVE_DECIMALS} to true to enable this.") } + + val types = wrapped.schema.map(_.dataType).toSet + if (types.exists(GpuOverrides.isOrContainsTimestamp(_))) { + if (!TimeZoneDB.isUTCTimezone()) { + willNotWorkOnGpu("Only UTC timezone is supported. " + + s"Current timezone settings: (JVM : ${ZoneId.systemDefault()}, " + + s"session: ${SQLConf.get.sessionLocalTimeZone}). ") + } + } } override def convertToGpu(): GpuExec = { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala index 737c74ca619..522e859d9ed 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala @@ -83,7 +83,7 @@ object GpuOrcFileFormat extends Logging { // [[org.apache.spark.sql.execution.datasources.DaysWritable]] object // which is a subclass of [[org.apache.hadoop.hive.serde2.io.DateWritable]]. val types = schema.map(_.dataType).toSet - if (types.exists(GpuOverrides.isContainsDateOrTimestamp(_))) { + if (types.exists(GpuOverrides.isOrContainsDateOrTimestamp(_))) { if (!TimeZoneDB.isUTCTimezone()) { meta.willNotWorkOnGpu("Only UTC timezone is supported for ORC. " + s"Current timezone settings: (JVM : ${ZoneId.systemDefault()}, " + From e62ee8074c433f56a03fcf539bd576e5d953e951 Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Tue, 28 Nov 2023 11:25:34 +0330 Subject: [PATCH 21/46] Change xfail to allow_non_gpu Signed-off-by: Chong Gao --- integration_tests/src/main/python/aqe_test.py | 3 +- .../src/main/python/cast_test.py | 30 ++++---- integration_tests/src/main/python/cmp_test.py | 4 +- .../src/main/python/collection_ops_test.py | 12 ++-- .../src/main/python/conditionals_test.py | 6 +- integration_tests/src/main/python/csv_test.py | 16 ++--- integration_tests/src/main/python/data_gen.py | 6 +- .../src/main/python/datasourcev2_read_test.py | 3 +- .../src/main/python/date_time_test.py | 68 +++++++++---------- .../src/main/python/explain_test.py | 2 +- .../src/main/python/generate_expr_test.py | 20 +++--- .../src/main/python/hash_aggregate_test.py | 44 ++++++------ .../main/python/hive_delimited_text_test.py | 12 ++-- .../src/main/python/hive_write_test.py | 2 +- .../src/main/python/join_test.py | 58 ++++++++-------- .../src/main/python/json_test.py | 29 ++++---- integration_tests/src/main/python/map_test.py | 32 ++++----- .../src/main/python/orc_cast_test.py | 9 +-- integration_tests/src/main/python/orc_test.py | 16 ++--- .../src/main/python/orc_write_test.py | 15 ++-- .../src/main/python/parquet_test.py | 10 +-- .../src/main/python/parquet_testing_test.py | 5 +- .../src/main/python/parquet_write_test.py | 30 ++++---- .../src/main/python/qa_nightly_select_test.py | 8 +-- .../src/main/python/repart_test.py | 10 +-- .../src/main/python/sort_test.py | 14 ++-- .../src/main/python/window_function_test.py | 41 ++++++----- 27 files changed, 250 insertions(+), 255 deletions(-) diff --git a/integration_tests/src/main/python/aqe_test.py b/integration_tests/src/main/python/aqe_test.py index 189bef329d7..7f2f68c5880 100755 --- a/integration_tests/src/main/python/aqe_test.py +++ b/integration_tests/src/main/python/aqe_test.py @@ -194,9 +194,8 @@ def do_it(spark): # broadcast join. The bug currently manifests in Databricks, but could # theoretically show up in other Spark distributions @ignore_order(local=True) -@allow_non_gpu('BroadcastNestedLoopJoinExec', 'Cast', 'DateSub', *db_113_cpu_bnlj_join_allow) +@allow_non_gpu('BroadcastNestedLoopJoinExec', 'Cast', 'DateSub', *db_113_cpu_bnlj_join_allow, *non_utc_allow) @pytest.mark.parametrize('join', joins, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_aqe_join_reused_exchange_inequality_condition(spark_tmp_path, join): data_path = spark_tmp_path + '/PARQUET_DATA' def prep(spark): diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index dbb41b60bb7..0b829ba9c03 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -152,7 +152,7 @@ def test_cast_string_date_non_ansi(): StringGen('[0-9]{1,4}-[0-3][0-9]-[0-5][0-9][ |T][0-3][0-9]:[0-6][0-9]:[0-6][0-9]'), StringGen('[0-9]{1,4}-[0-3][0-9]-[0-5][0-9][ |T][0-3][0-9]:[0-6][0-9]:[0-6][0-9].[0-9]{0,6}Z?')], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_string_ts_valid_format(data_gen): # In Spark 3.2.0+ the valid format changed, and we cannot support all of the format. # This provides values that are valid in all of those formats. @@ -300,7 +300,7 @@ def _assert_cast_to_string_equal (data_gen, conf): @pytest.mark.parametrize('data_gen', all_array_gens_for_cast_to_string, ids=idfn) @pytest.mark.parametrize('legacy', ['true', 'false']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_array_to_string(data_gen, legacy): _assert_cast_to_string_equal( data_gen, @@ -320,7 +320,7 @@ def test_cast_array_with_unmatched_element_to_string(data_gen, legacy): @pytest.mark.parametrize('data_gen', basic_map_gens_for_cast_to_string, ids=idfn) @pytest.mark.parametrize('legacy', ['true', 'false']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_map_to_string(data_gen, legacy): _assert_cast_to_string_equal( data_gen, @@ -340,7 +340,7 @@ def test_cast_map_with_unmatched_element_to_string(data_gen, legacy): @pytest.mark.parametrize('data_gen', [StructGen([[str(i), gen] for i, gen in enumerate(basic_array_struct_gens_for_cast_to_string)] + [["map", MapGen(ByteGen(nullable=False), null_gen)]])], ids=idfn) @pytest.mark.parametrize('legacy', ['true', 'false']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_struct_to_string(data_gen, legacy): _assert_cast_to_string_equal( data_gen, @@ -405,7 +405,7 @@ def test_cast_string_to_negative_scale_decimal(): @pytest.mark.skipif(is_before_spark_330(), reason="ansi cast throws exception only in 3.3.0+") @pytest.mark.parametrize('type', [DoubleType(), FloatType()], ids=idfn) @pytest.mark.parametrize('invalid_value', [float("inf"), float("-inf"), float("nan")]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_float_to_timestamp_ansi_for_nan_inf(type, invalid_value): def fun(spark): data = [invalid_value] @@ -417,7 +417,7 @@ def fun(spark): @pytest.mark.skipif(is_before_spark_330(), reason="ansi cast throws exception only in 3.3.0+") @pytest.mark.parametrize('type', [DoubleType(), FloatType()], ids=idfn) @pytest.mark.parametrize('invalid_value', [float(LONG_MAX) + 100, float(LONG_MIN) - 100]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_float_to_timestamp_ansi_overflow(type, invalid_value): def fun(spark): data = [invalid_value] @@ -426,7 +426,7 @@ def fun(spark): assert_gpu_and_cpu_error(fun, {"spark.sql.ansi.enabled": True}, "ArithmeticException") @pytest.mark.skipif(is_before_spark_330(), reason='330+ throws exception in ANSI mode') -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_float_to_timestamp_side_effect(): def getDf(spark): data = [(True, float(LONG_MAX) + 100), (False, float(1))] @@ -438,7 +438,7 @@ def getDf(spark): # non ansi mode, will get null @pytest.mark.parametrize('type', [DoubleType(), FloatType()], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_float_to_timestamp_for_nan_inf(type): def fun(spark): data = [(float("inf"),), (float("-inf"),), (float("nan"),)] @@ -458,7 +458,7 @@ def fun(spark): short_gen, int_gen, long_gen_to_timestamp], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_integral_to_timestamp(gen, ansi_enabled): if(is_before_spark_330() and ansi_enabled): # 330- does not support in ANSI mode pytest.skip() @@ -467,7 +467,7 @@ def test_cast_integral_to_timestamp(gen, ansi_enabled): conf={"spark.sql.ansi.enabled": ansi_enabled}) @pytest.mark.parametrize('ansi_enabled', [True, False], ids=['ANSI_ON', 'ANSI_OFF']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_float_to_timestamp(ansi_enabled): if(is_before_spark_330() and ansi_enabled): # 330- does not support in ANSI mode pytest.skip() @@ -477,7 +477,7 @@ def test_cast_float_to_timestamp(ansi_enabled): conf={"spark.sql.ansi.enabled": ansi_enabled}) @pytest.mark.parametrize('ansi_enabled', [True, False], ids=['ANSI_ON', 'ANSI_OFF']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_double_to_timestamp(ansi_enabled): if (is_before_spark_330() and ansi_enabled): # 330- does not support in ANSI mode pytest.skip() @@ -495,7 +495,7 @@ def test_cast_double_to_timestamp(ansi_enabled): (INT_MIN - 1, IntegerType()), ], ids=idfn) @pytest.mark.skipif(is_before_spark_330(), reason="Spark 330- does not ansi casting between numeric and timestamp") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_timestamp_to_integral_ansi_overflow(invalid_and_type): (invalid, to_type) = invalid_and_type assert_gpu_and_cpu_error( @@ -506,7 +506,7 @@ def test_cast_timestamp_to_integral_ansi_overflow(invalid_and_type): error_message="overflow") @pytest.mark.skipif(is_before_spark_330(), reason="Spark 330- does not ansi casting between numeric and timestamp") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_timestamp_to_numeric_ansi_no_overflow(): data = [datetime.fromtimestamp(i) for i in range(BYTE_MIN, BYTE_MAX + 1)] assert_gpu_and_cpu_are_equal_collect( @@ -515,14 +515,14 @@ def test_cast_timestamp_to_numeric_ansi_no_overflow(): "cast(value as float)", "cast(value as double)"), conf=ansi_enabled_conf) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_timestamp_to_numeric_non_ansi(): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, timestamp_gen) .selectExpr("cast(a as byte)", "cast(a as short)", "cast(a as int)", "cast(a as long)", "cast(a as float)", "cast(a as double)")) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cast_timestamp_to_string(): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, timestamp_gen) diff --git a/integration_tests/src/main/python/cmp_test.py b/integration_tests/src/main/python/cmp_test.py index b451f9c3db7..8a92c268182 100644 --- a/integration_tests/src/main/python/cmp_test.py +++ b/integration_tests/src/main/python/cmp_test.py @@ -19,7 +19,7 @@ from data_gen import * from spark_session import with_cpu_session, is_before_spark_330 from pyspark.sql.types import * -from marks import datagen_overrides +from marks import datagen_overrides, allow_non_gpu import pyspark.sql.functions as f @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + struct_gens_sample_with_decimal128_no_list, ids=idfn) @@ -336,7 +336,7 @@ def test_in(data_gen): # This is to test entries over that value. @datagen_overrides(seed=0, reason='https://github.com/NVIDIA/spark-rapids/issues/9687') @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_in_set(data_gen): # nulls are not supported for in on the GPU yet num_entries = int(with_cpu_session(lambda spark: spark.conf.get('spark.sql.optimizer.inSetConversionThreshold'))) + 1 diff --git a/integration_tests/src/main/python/collection_ops_test.py b/integration_tests/src/main/python/collection_ops_test.py index 04841e7f3ff..26810a223fa 100644 --- a/integration_tests/src/main/python/collection_ops_test.py +++ b/integration_tests/src/main/python/collection_ops_test.py @@ -23,6 +23,8 @@ import pyspark.sql.utils from spark_session import with_cpu_session, with_gpu_session from conftest import get_datagen_seed +from marks import allow_non_gpu + nested_gens = [ArrayGen(LongGen()), ArrayGen(decimal_gen_128bit), StructGen([("a", LongGen()), ("b", decimal_gen_128bit)]), @@ -251,7 +253,7 @@ def test_sort_array_normalize_nans(): gens in sequence_normal_integral_gens] @pytest.mark.parametrize('start_gen,stop_gen', sequence_normal_no_step_integral_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_sequence_without_step(start_gen, stop_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: two_col_df(spark, start_gen, stop_gen).selectExpr( @@ -260,7 +262,7 @@ def test_sequence_without_step(start_gen, stop_gen): "sequence(20, b)")) @pytest.mark.parametrize('start_gen,stop_gen,step_gen', sequence_normal_integral_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_sequence_with_step(start_gen, stop_gen, step_gen): # Get the datagen seed we use for all datagens, since we need to call start # on step_gen @@ -307,7 +309,7 @@ def test_sequence_with_step(start_gen, stop_gen, step_gen): ] @pytest.mark.parametrize('start_gen,stop_gen,step_gen', sequence_illegal_boundaries_integral_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_sequence_illegal_boundaries(start_gen, stop_gen, step_gen): assert_gpu_and_cpu_error( lambda spark:three_col_df(spark, start_gen, stop_gen, step_gen).selectExpr( @@ -322,7 +324,7 @@ def test_sequence_illegal_boundaries(start_gen, stop_gen, step_gen): ] @pytest.mark.parametrize('stop_gen', sequence_too_long_length_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_sequence_too_long_sequence(stop_gen): assert_gpu_and_cpu_error( # To avoid OOM, reduce the row number to 1, it is enough to verify this case. @@ -364,7 +366,7 @@ def get_sequence_data(gen, len): mixed_schema) # test for 3 cases mixed in a single dataset -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_sequence_with_step_mixed_cases(): assert_gpu_and_cpu_are_equal_collect( lambda spark: get_sequence_cases_mixed_df(spark) diff --git a/integration_tests/src/main/python/conditionals_test.py b/integration_tests/src/main/python/conditionals_test.py index 1bc11c5d27b..b418483fb10 100644 --- a/integration_tests/src/main/python/conditionals_test.py +++ b/integration_tests/src/main/python/conditionals_test.py @@ -19,7 +19,7 @@ from data_gen import * from spark_session import is_before_spark_320, is_jvm_charset_utf8 from pyspark.sql.types import * -from marks import datagen_overrides +from marks import datagen_overrides, allow_non_gpu import pyspark.sql.functions as f def mk_str_gen(pattern): @@ -233,7 +233,7 @@ def test_conditional_with_side_effects_case_when(data_gen): conf = test_conf) @pytest.mark.parametrize('data_gen', [mk_str_gen('[a-z]{0,3}')], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_conditional_with_side_effects_sequence(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr( @@ -244,7 +244,7 @@ def test_conditional_with_side_effects_sequence(data_gen): @pytest.mark.skipif(is_before_spark_320(), reason='Earlier versions of Spark cannot cast sequence to string') @pytest.mark.parametrize('data_gen', [mk_str_gen('[a-z]{0,3}')], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_conditional_with_side_effects_sequence_cast(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr( diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index c10221a4407..c7330a06c33 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -248,7 +248,7 @@ def read_impl(spark): @pytest.mark.parametrize('read_func', [read_csv_df, read_csv_sql]) @pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_basic_csv_read(std_input_path, name, schema, options, read_func, v1_enabled_list, ansi_enabled, spark_tmp_table_factory): updated_conf=copy_and_update(_enable_all_types_conf, { 'spark.sql.sources.useV1SourceList': v1_enabled_list, @@ -289,7 +289,7 @@ def test_csv_read_small_floats(std_input_path, name, schema, options, read_func, @approximate_float @pytest.mark.parametrize('data_gen', csv_supported_gens, ids=idfn) @pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_round_trip(spark_tmp_path, data_gen, v1_enabled_list): gen = StructGen([('a', data_gen)], nullable=False) data_path = spark_tmp_path + '/CSV_DATA' @@ -406,7 +406,7 @@ def test_read_valid_and_invalid_dates(std_input_path, filename, v1_enabled_list, @pytest.mark.parametrize('ts_part', csv_supported_ts_parts) @pytest.mark.parametrize('date_format', csv_supported_date_formats) @pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_ts_formats_round_trip(spark_tmp_path, date_format, ts_part, v1_enabled_list): full_format = date_format + ts_part data_gen = TimestampGen() @@ -476,8 +476,7 @@ def test_input_meta_fallback(spark_tmp_path, v1_enabled_list, disable_conf): cpu_fallback_class_name = 'FileSourceScanExec' if v1_enabled_list == 'csv' else 'BatchScanExec', conf=updated_conf) -@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec') -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec', *non_utc_allow) def test_csv_save_as_table_fallback(spark_tmp_path, spark_tmp_table_factory): gen = TimestampGen() data_path = spark_tmp_path + '/CSV_DATA' @@ -563,14 +562,13 @@ def test_csv_read_count(spark_tmp_path): assert_gpu_and_cpu_row_counts_equal(lambda spark: spark.read.csv(data_path), conf = {'spark.rapids.sql.explain': 'ALL'}) -@allow_non_gpu('FileSourceScanExec', 'ProjectExec', 'CollectLimitExec', 'DeserializeToObjectExec') +@allow_non_gpu('FileSourceScanExec', 'ProjectExec', 'CollectLimitExec', 'DeserializeToObjectExec', *non_utc_allow) @pytest.mark.skipif(is_before_spark_341(), reason='`TIMESTAMP_NTZ` is only supported in PySpark 341+') @pytest.mark.parametrize('date_format', csv_supported_date_formats) @pytest.mark.parametrize('ts_part', csv_supported_ts_parts) @pytest.mark.parametrize("timestamp_type", [ pytest.param('TIMESTAMP_LTZ', marks=pytest.mark.xfail(is_spark_350_or_later(), reason="https://github.com/NVIDIA/spark-rapids/issues/9325")), "TIMESTAMP_NTZ"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_csv_infer_schema_timestamp_ntz_v1(spark_tmp_path, date_format, ts_part, timestamp_type): csv_infer_schema_timestamp_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, 'csv', 'FileSourceScanExec') @@ -621,9 +619,9 @@ def do_read(spark): non_exist_classes = cpu_scan_class, conf = conf) -@allow_non_gpu('FileSourceScanExec', 'CollectLimitExec', 'DeserializeToObjectExec') +@allow_non_gpu('FileSourceScanExec', 'CollectLimitExec', 'DeserializeToObjectExec', *non_utc_allow) @pytest.mark.skipif(is_before_spark_340(), reason='`preferDate` is only supported in Spark 340+') -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') + def test_csv_prefer_date_with_infer_schema(spark_tmp_path): # start date ""0001-01-02" required due to: https://github.com/NVIDIA/spark-rapids/issues/5606 data_gens = [byte_gen, short_gen, int_gen, long_gen, boolean_gen, timestamp_gen, DateGen(start=date(1, 1, 2))] diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index 9aa5e547c45..ca2619ca0e3 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -24,7 +24,7 @@ from spark_session import is_before_spark_340, with_cpu_session import sre_yield import struct -from conftest import skip_unless_precommit_tests,get_datagen_seed +from conftest import skip_unless_precommit_tests,get_datagen_seed, is_not_utc import time import os from functools import lru_cache @@ -1172,3 +1172,7 @@ def get_25_partitions_df(spark): StructField("c3", IntegerType())]) data = [[i, j, k] for i in range(0, 5) for j in range(0, 5) for k in range(0, 100)] return spark.createDataFrame(data, schema) + + +# allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' +non_utc_allow=['ProjectExec', 'FilterExec', 'FileSourceScanExec', 'BatchScanExec', 'CollectLimitExec', 'DeserializeToObjectExec', 'DataWritingCommandExec', 'WriteFilesExec', 'ShuffleExchangeExec', 'ObjectHashAggregateExec'] if is_not_utc() else [] \ No newline at end of file diff --git a/integration_tests/src/main/python/datasourcev2_read_test.py b/integration_tests/src/main/python/datasourcev2_read_test.py index 4a25d618e7d..b2134c50618 100644 --- a/integration_tests/src/main/python/datasourcev2_read_test.py +++ b/integration_tests/src/main/python/datasourcev2_read_test.py @@ -16,6 +16,7 @@ from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_row_counts_equal from conftest import is_not_utc +from data_gen import * from marks import * columnarClass = 'com.nvidia.spark.rapids.tests.datasourcev2.parquet.ArrowColumnarDataSourceV2' @@ -31,7 +32,7 @@ def test_read_int(): assert_gpu_and_cpu_are_equal_collect(readTable("int", columnarClass)) @validate_execs_in_gpu_plan('HostColumnarToGpu') -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_read_strings(): assert_gpu_and_cpu_are_equal_collect(readTable("string", columnarClass)) diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index 1d984193f9e..1ff3512fdbe 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -26,7 +26,7 @@ vals = [(-584, 1563), (1943, 1101), (2693, 2167), (2729, 0), (44, 1534), (2635, 3319), (1885, -2828), (0, 2463), (932, 2286), (0, 0)] @pytest.mark.parametrize('data_gen', vals, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timesub(data_gen): days, seconds = data_gen assert_gpu_and_cpu_are_equal_collect( @@ -35,7 +35,7 @@ def test_timesub(data_gen): .selectExpr("a - (interval {} days {} seconds)".format(days, seconds))) @pytest.mark.parametrize('data_gen', vals, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timeadd(data_gen): days, seconds = data_gen assert_gpu_and_cpu_are_equal_collect( @@ -45,7 +45,7 @@ def test_timeadd(data_gen): .selectExpr("a + (interval {} days {} seconds)".format(days, seconds))) @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timeadd_daytime_column(): gen_list = [ # timestamp column max year is 1000 @@ -63,7 +63,7 @@ def test_interval_seconds_overflow_exception(): error_message="IllegalArgumentException") @pytest.mark.parametrize('data_gen', vals, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timeadd_from_subquery(data_gen): def fun(spark): @@ -75,7 +75,7 @@ def fun(spark): assert_gpu_and_cpu_are_equal_collect(fun) @pytest.mark.parametrize('data_gen', vals, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timesub_from_subquery(data_gen): def fun(spark): @@ -91,7 +91,7 @@ def fun(spark): # [SPARK-34896][SQL] Return day-time interval from dates subtraction # 1. Add the SQL config `spark.sql.legacy.interval.enabled` which will control when Spark SQL should use `CalendarIntervalType` instead of ANSI intervals. @pytest.mark.parametrize('data_gen', vals, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_dateaddinterval(data_gen): days, seconds = data_gen assert_gpu_and_cpu_are_equal_collect( @@ -102,7 +102,7 @@ def test_dateaddinterval(data_gen): # test add days(not specify hours, minutes, seconds, milliseconds, microseconds) in ANSI mode. @pytest.mark.parametrize('data_gen', vals, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_dateaddinterval_ansi(data_gen): days, _ = data_gen # only specify the `days` @@ -130,17 +130,17 @@ def test_datediff(data_gen): 'datediff(a, date(null))', 'datediff(a, \'2016-03-02\')')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hour(): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, timestamp_gen).selectExpr('hour(a)')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_minute(): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, timestamp_gen).selectExpr('minute(a)')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_second(): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, timestamp_gen).selectExpr('second(a)')) @@ -199,7 +199,7 @@ def test_datesub(data_gen): to_unix_timestamp_days_gen=[ByteGen(), ShortGen(), IntegerGen(min_val=-106032829, max_val=103819094, special_cases=[-106032829, 103819094,0,1,-1])] @pytest.mark.parametrize('data_gen', to_unix_timestamp_days_gen, ids=idfn) @incompat -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_dateadd_with_date_overflow(data_gen): string_type = to_cast_string(data_gen.data_type) assert_gpu_and_cpu_are_equal_collect( @@ -213,7 +213,7 @@ def test_dateadd_with_date_overflow(data_gen): to_unix_timestamp_days_gen=[ByteGen(), ShortGen(), IntegerGen(max_val=106032829, min_val=-103819094, special_cases=[106032829, -103819094,0,1,-1])] @pytest.mark.parametrize('data_gen', to_unix_timestamp_days_gen, ids=idfn) @incompat -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_datesub_with_date_overflow(data_gen): string_type = to_cast_string(data_gen.data_type) assert_gpu_and_cpu_are_equal_collect( @@ -245,7 +245,7 @@ def test_dayofyear(data_gen): lambda spark : unary_op_df(spark, data_gen).select(f.dayofyear(f.col('a')))) @pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_unix_timestamp(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).select(f.unix_timestamp(f.col('a')))) @@ -262,7 +262,7 @@ def test_unsupported_fallback_unix_timestamp(data_gen): @pytest.mark.parametrize('ansi_enabled', [True, False], ids=['ANSI_ON', 'ANSI_OFF']) @pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_to_unix_timestamp(data_gen, ansi_enabled): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr("to_unix_timestamp(a)"), @@ -280,7 +280,7 @@ def test_unsupported_fallback_to_unix_timestamp(data_gen): @pytest.mark.parametrize('time_zone', ["UTC", "UTC+0", "UTC-0", "GMT", "GMT+0", "GMT-0"], ids=idfn) @pytest.mark.parametrize('data_gen', [timestamp_gen], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_from_utc_timestamp(data_gen, time_zone): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).select(f.from_utc_timestamp(f.col('a'), time_zone))) @@ -296,7 +296,7 @@ def test_from_utc_timestamp_unsupported_timezone_fallback(data_gen, time_zone): @pytest.mark.parametrize('time_zone', ["UTC", "Asia/Shanghai", "EST", "MST", "VST"], ids=idfn) @pytest.mark.parametrize('data_gen', [timestamp_gen], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_from_utc_timestamp_supported_timezones(data_gen, time_zone): # Remove spark.rapids.test.CPU.timezone configuration when GPU kernel is ready to really test on GPU assert_gpu_and_cpu_are_equal_collect( @@ -354,7 +354,7 @@ def fun(spark): @pytest.mark.parametrize('parser_policy', ["CORRECTED", "EXCEPTION"], ids=idfn) # first get expected string via `date_format` -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_string_to_timestamp_functions_ansi_valid(parser_policy): expr_format = "{operator}(date_format(a, '{fmt}'), '{fmt}')" formats = ['yyyy-MM-dd', 'yyyy/MM/dd', 'yyyy-MM', 'yyyy/MM', 'dd/MM/yyyy', 'yyyy-MM-dd HH:mm:ss', @@ -372,7 +372,7 @@ def fun(spark): @pytest.mark.parametrize('ansi_enabled', [True, False], ids=['ANSI_ON', 'ANSI_OFF']) @pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_unix_timestamp_improved(data_gen, ansi_enabled): conf = {"spark.rapids.sql.improvedTimeOps.enabled": "true", "spark.sql.legacy.timeParserPolicy": "CORRECTED"} @@ -382,7 +382,7 @@ def test_unix_timestamp_improved(data_gen, ansi_enabled): @pytest.mark.parametrize('ansi_enabled', [True, False], ids=['ANSI_ON', 'ANSI_OFF']) @pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_unix_timestamp(data_gen, ansi_enabled): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).select(f.unix_timestamp(f.col("a"))), @@ -390,7 +390,7 @@ def test_unix_timestamp(data_gen, ansi_enabled): @pytest.mark.parametrize('ansi_enabled', [True, False], ids=['ANSI_ON', 'ANSI_OFF']) @pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_to_unix_timestamp_improved(data_gen, ansi_enabled): conf = {"spark.rapids.sql.improvedTimeOps.enabled": "true"} assert_gpu_and_cpu_are_equal_collect( @@ -409,7 +409,7 @@ def invalid_date_string_df(spark): @pytest.mark.parametrize('ansi_enabled', [True, False], ids=['ANSI_ON', 'ANSI_OFF']) @pytest.mark.parametrize('data_gen,date_form', str_date_and_format_gen, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_string_to_unix_timestamp(data_gen, date_form, ansi_enabled): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen, seed=1).selectExpr("to_unix_timestamp(a, '{}')".format(date_form)), @@ -423,7 +423,7 @@ def test_string_to_unix_timestamp_ansi_exception(): @pytest.mark.parametrize('ansi_enabled', [True, False], ids=['ANSI_ON', 'ANSI_OFF']) @pytest.mark.parametrize('data_gen,date_form', str_date_and_format_gen, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_string_unix_timestamp(data_gen, date_form, ansi_enabled): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen, seed=1).select(f.unix_timestamp(f.col('a'), date_form)), @@ -437,7 +437,7 @@ def test_string_unix_timestamp_ansi_exception(): @pytest.mark.parametrize('data_gen', [StringGen('200[0-9]-0[1-9]-[0-2][1-8]')], ids=idfn) @pytest.mark.parametrize('ansi_enabled', [True, False], ids=['ANSI_ON', 'ANSI_OFF']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_gettimestamp(data_gen, ansi_enabled): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).select(f.to_date(f.col("a"), "yyyy-MM-dd")), @@ -445,7 +445,7 @@ def test_gettimestamp(data_gen, ansi_enabled): @pytest.mark.parametrize('data_gen', [StringGen('0[1-9]200[0-9]')], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_gettimestamp_format_MMyyyy(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).select(f.to_date(f.col("a"), "MMyyyy"))) @@ -460,7 +460,7 @@ def test_gettimestamp_ansi_exception(): 'MM-dd', 'MM/dd', 'dd-MM', 'dd/MM'] @pytest.mark.parametrize('date_format', supported_date_formats, ids=idfn) @pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_date_format(data_gen, date_format): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr("date_format(a, '{}')".format(date_format))) @@ -495,7 +495,7 @@ def test_date_format_maybe(data_gen, date_format): @pytest.mark.parametrize('date_format', maybe_supported_date_formats, ids=idfn) @pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_date_format_maybe_incompat(data_gen, date_format): conf = {"spark.rapids.sql.incompatibleDateFormats.enabled": "true"} assert_gpu_and_cpu_are_equal_collect( @@ -507,7 +507,7 @@ def test_date_format_maybe_incompat(data_gen, date_format): # input_file_name(), otherwise filter happens before project. @allow_non_gpu('CollectLimitExec,FileSourceScanExec,DeserializeToObjectExec') @ignore_order() -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_date_format_mmyyyy_cast_canonicalization(spark_tmp_path): data_path = spark_tmp_path + '/CSV_DATA' gen = StringGen(pattern='[0][0-9][1][8-9][1-9][1-9]', nullable=False) @@ -553,12 +553,12 @@ def test_unsupported_fallback_to_date(): seconds_gens = [LongGen(min_val=-62135510400, max_val=253402214400), IntegerGen(), ShortGen(), ByteGen(), DoubleGen(min_exp=0, max_exp=32), ts_float_gen, DecimalGen(16, 6), DecimalGen(13, 3), DecimalGen(10, 0), DecimalGen(7, -3), DecimalGen(6, 6)] @pytest.mark.parametrize('data_gen', seconds_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timestamp_seconds(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr("timestamp_seconds(a)")) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timestamp_seconds_long_overflow(): assert_gpu_and_cpu_error( lambda spark : unary_op_df(spark, long_gen).selectExpr("timestamp_seconds(a)").collect(), @@ -566,7 +566,7 @@ def test_timestamp_seconds_long_overflow(): error_message='long overflow') @pytest.mark.parametrize('data_gen', [DecimalGen(7, 7), DecimalGen(20, 7)], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timestamp_seconds_rounding_necessary(data_gen): assert_gpu_and_cpu_error( lambda spark : unary_op_df(spark, data_gen).selectExpr("timestamp_seconds(a)").collect(), @@ -574,7 +574,7 @@ def test_timestamp_seconds_rounding_necessary(data_gen): error_message='Rounding necessary') @pytest.mark.parametrize('data_gen', [DecimalGen(19, 6), DecimalGen(20, 6)], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timestamp_seconds_decimal_overflow(data_gen): assert_gpu_and_cpu_error( lambda spark : unary_op_df(spark, data_gen).selectExpr("timestamp_seconds(a)").collect(), @@ -583,12 +583,12 @@ def test_timestamp_seconds_decimal_overflow(data_gen): millis_gens = [LongGen(min_val=-62135510400000, max_val=253402214400000), IntegerGen(), ShortGen(), ByteGen()] @pytest.mark.parametrize('data_gen', millis_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timestamp_millis(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr("timestamp_millis(a)")) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timestamp_millis_long_overflow(): assert_gpu_and_cpu_error( lambda spark : unary_op_df(spark, long_gen).selectExpr("timestamp_millis(a)").collect(), @@ -597,7 +597,7 @@ def test_timestamp_millis_long_overflow(): micros_gens = [LongGen(min_val=-62135510400000000, max_val=253402214400000000), IntegerGen(), ShortGen(), ByteGen()] @pytest.mark.parametrize('data_gen', micros_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timestamp_micros(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr("timestamp_micros(a)")) diff --git a/integration_tests/src/main/python/explain_test.py b/integration_tests/src/main/python/explain_test.py index 1837f31aa95..d182c093850 100644 --- a/integration_tests/src/main/python/explain_test.py +++ b/integration_tests/src/main/python/explain_test.py @@ -50,7 +50,7 @@ def do_join_explain(spark): with_cpu_session(do_join_explain) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_explain_set_config(): conf = {'spark.rapids.sql.hasExtendedYearValues': 'false', 'spark.rapids.sql.castStringToTimestamp.enabled': 'true'} diff --git a/integration_tests/src/main/python/generate_expr_test.py b/integration_tests/src/main/python/generate_expr_test.py index acc3e125ee6..7e8c5ced399 100644 --- a/integration_tests/src/main/python/generate_expr_test.py +++ b/integration_tests/src/main/python/generate_expr_test.py @@ -115,7 +115,7 @@ def test_explode_outer_map_data(map_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_explode_outer_nested_array_data(data_gen): data_gen = [int_gen, ArrayGen(ArrayGen(data_gen))] assert_gpu_and_cpu_are_equal_collect( @@ -127,7 +127,7 @@ def test_explode_outer_nested_array_data(data_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_posexplode_makearray(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : four_op_df(spark, data_gen).selectExpr('posexplode(array(b, c, d))', 'a')) @@ -136,7 +136,7 @@ def test_posexplode_makearray(data_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_posexplode_litarray(data_gen): array_lit = with_cpu_session( lambda spark: gen_scalar(ArrayGen(data_gen, min_length=3, max_length=3, nullable=False))) @@ -151,7 +151,7 @@ def test_posexplode_litarray(data_gen): @pytest.mark.parametrize('data_gen', explode_gens + struct_gens_sample_with_decimal128 + array_gens_sample + arrays_with_binary + map_gens_sample + maps_with_binary, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_posexplode_array_data(data_gen): data_gen = [int_gen, ArrayGen(data_gen)] assert_gpu_and_cpu_are_equal_collect( @@ -162,7 +162,7 @@ def test_posexplode_array_data(data_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('map_gen', map_gens_sample + decimal_128_map_gens + maps_with_binary, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_posexplode_map_data(map_gen): data_gen = [int_gen, map_gen] assert_gpu_and_cpu_are_equal_collect( @@ -173,7 +173,7 @@ def test_posexplode_map_data(map_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_posexplode_nested_array_data(data_gen): data_gen = [int_gen, ArrayGen(ArrayGen(data_gen))] assert_gpu_and_cpu_are_equal_collect( @@ -188,7 +188,7 @@ def test_posexplode_nested_array_data(data_gen): @pytest.mark.parametrize('data_gen', explode_gens + struct_gens_sample_with_decimal128 + array_gens_sample + arrays_with_binary + map_gens_sample + maps_with_binary, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_posexplode_outer_array_data(data_gen): data_gen = [int_gen, ArrayGen(data_gen)] assert_gpu_and_cpu_are_equal_collect( @@ -199,7 +199,7 @@ def test_posexplode_outer_array_data(data_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('map_gen', map_gens_sample + decimal_128_map_gens + maps_with_binary, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_posexplode_outer_map_data(map_gen): data_gen = [int_gen, map_gen] assert_gpu_and_cpu_are_equal_collect( @@ -210,7 +210,7 @@ def test_posexplode_outer_map_data(map_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_posexplode_nested_outer_array_data(data_gen): data_gen = [int_gen, ArrayGen(ArrayGen(data_gen))] assert_gpu_and_cpu_are_equal_collect( @@ -235,7 +235,7 @@ def test_stack(): # gpu stack not guarantee to produce the same output order as Spark does @ignore_order(local=True) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_stack_mixed_types(): base_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, date_gen, timestamp_gen, null_gen, DecimalGen(precision=7, scale=3), diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 03422e3f4bc..a6f1bb017fe 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -631,7 +631,7 @@ def test_min_max_group_by(data_gen): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _gen_data_for_collect_list_op, ids=idfn) @pytest.mark.parametrize('use_obj_hash_agg', [True, False], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_groupby_collect_list(data_gen, use_obj_hash_agg): def doit(spark): df = gen_df(spark, data_gen, length=100)\ @@ -663,7 +663,7 @@ def doit(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _full_gen_data_for_collect_op, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_groupby_collect_set(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=100) @@ -672,7 +672,7 @@ def test_hash_groupby_collect_set(data_gen): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _gen_data_for_collect_set_op, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_groupby_collect_set_on_nested_type(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=100) @@ -687,7 +687,7 @@ def test_hash_groupby_collect_set_on_nested_type(data_gen): @ignore_order(local=True) @allow_non_gpu("ProjectExec", "SortArray") @pytest.mark.parametrize('data_gen', _gen_data_for_collect_set_op_nested, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_groupby_collect_set_on_nested_array_type(data_gen): conf = copy_and_update(_float_conf, { "spark.rapids.sql.castFloatToString.enabled": "true", @@ -709,7 +709,7 @@ def do_it(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _full_gen_data_for_collect_op, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_reduction_collect_set(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=100) @@ -717,7 +717,7 @@ def test_hash_reduction_collect_set(data_gen): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _gen_data_for_collect_set_op, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_reduction_collect_set_on_nested_type(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=100) @@ -731,7 +731,7 @@ def test_hash_reduction_collect_set_on_nested_type(data_gen): @ignore_order(local=True) @allow_non_gpu("ProjectExec", "SortArray") @pytest.mark.parametrize('data_gen', _gen_data_for_collect_set_op_nested, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_reduction_collect_set_on_nested_array_type(data_gen): conf = copy_and_update(_float_conf, { "spark.rapids.sql.castFloatToString.enabled": "true", @@ -751,7 +751,7 @@ def do_it(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _full_gen_data_for_collect_op, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_groupby_collect_with_single_distinct(data_gen): # test collect_ops with other distinct aggregations assert_gpu_and_cpu_are_equal_collect( @@ -764,7 +764,7 @@ def test_hash_groupby_collect_with_single_distinct(data_gen): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _gen_data_for_collect_op, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_groupby_single_distinct_collect(data_gen): # test distinct collect sql = """select a, @@ -788,7 +788,7 @@ def test_hash_groupby_single_distinct_collect(data_gen): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _gen_data_for_collect_op, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_groupby_collect_with_multi_distinct(data_gen): def spark_fn(spark_session): return gen_df(spark_session, data_gen, length=100).groupby('a').agg( @@ -815,7 +815,7 @@ def spark_fn(spark_session): @pytest.mark.parametrize('replace_mode', _replace_modes_non_distinct, ids=idfn) @pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn) @pytest.mark.parametrize('use_obj_hash_agg', ['false', 'true'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_groupby_collect_partial_replace_fallback(data_gen, replace_mode, aqe_enabled, @@ -863,7 +863,7 @@ def test_hash_groupby_collect_partial_replace_fallback(data_gen, @pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn) @pytest.mark.parametrize('use_obj_hash_agg', ['false', 'true'], ids=idfn) @pytest.mark.xfail(condition=is_databricks104_or_later(), reason='https://github.com/NVIDIA/spark-rapids/issues/4963') -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_groupby_collect_partial_replace_with_distinct_fallback(data_gen, replace_mode, aqe_enabled, @@ -1262,7 +1262,7 @@ def test_first_last_reductions_decimal_types(data_gen): 'first(a)', 'last(a)', 'first(a, true)', 'last(a, true)')) @pytest.mark.parametrize('data_gen', _nested_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_first_last_reductions_nested_types(data_gen): assert_gpu_and_cpu_are_equal_collect( # Coalesce and sort are to make sure that first and last, which are non-deterministic @@ -1271,7 +1271,7 @@ def test_first_last_reductions_nested_types(data_gen): 'first(a)', 'last(a)', 'first(a, true)', 'last(a, true)')) @pytest.mark.parametrize('data_gen', _all_basic_gens_with_all_nans_cases, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_generic_reductions(data_gen): local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'}) assert_gpu_and_cpu_are_equal_collect( @@ -1289,7 +1289,7 @@ def test_generic_reductions(data_gen): conf=local_conf) @pytest.mark.parametrize('data_gen', all_gen + _nested_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_count(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen) \ @@ -1301,7 +1301,7 @@ def test_count(data_gen): conf = {'spark.sql.legacy.allowParameterlessCount': 'true'}) @pytest.mark.parametrize('data_gen', all_basic_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_distinct_count_reductions(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).selectExpr( @@ -1325,7 +1325,7 @@ def test_arithmetic_reductions(data_gen): @pytest.mark.parametrize('data_gen', all_basic_gens + decimal_gens + _nested_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_collect_list_reductions(data_gen): assert_gpu_and_cpu_are_equal_collect( # coalescing because collect_list is not deterministic @@ -1344,7 +1344,7 @@ def test_collect_list_reductions(data_gen): @pytest.mark.parametrize('data_gen', _no_neg_zero_all_basic_gens + decimal_gens + _struct_only_nested_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_collect_set_reductions(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr('sort_array(collect_set(a))'), @@ -1358,7 +1358,7 @@ def test_collect_empty(): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', all_gen + _nested_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_groupby_first_last(data_gen): gen_fn = [('a', RepeatSeqGen(LongGen(), length=20)), ('b', data_gen)] agg_fn = lambda df: df.groupBy('a').agg( @@ -1372,7 +1372,7 @@ def test_groupby_first_last(data_gen): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', all_gen + _struct_only_nested_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_sorted_groupby_first_last(data_gen): gen_fn = [('a', RepeatSeqGen(LongGen(), length=20)), ('b', data_gen)] # sort by more than the group by columns to be sure that first/last don't remove the ordering @@ -1390,7 +1390,7 @@ def test_sorted_groupby_first_last(data_gen): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', all_gen, ids=idfn) @pytest.mark.parametrize('count_func', [f.count, f.countDistinct]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_agg_count(data_gen, count_func): assert_gpu_and_cpu_are_equal_collect( lambda spark : gen_df(spark, [('a', data_gen), ('b', data_gen)], @@ -2047,7 +2047,7 @@ def test_std_variance_partial_replace_fallback(data_gen, null_gen] + array_gens_sample + struct_gens_sample @ignore_order(local=True) @pytest.mark.parametrize('data_gen', gens_for_max_min, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_min_max_in_groupby_and_reduction(data_gen): df_gen = [('a', data_gen), ('b', RepeatSeqGen(IntegerGen(), length=20))] diff --git a/integration_tests/src/main/python/hive_delimited_text_test.py b/integration_tests/src/main/python/hive_delimited_text_test.py index 4d07a077ec0..78c77391c31 100644 --- a/integration_tests/src/main/python/hive_delimited_text_test.py +++ b/integration_tests/src/main/python/hive_delimited_text_test.py @@ -187,7 +187,7 @@ def read_impl(spark): ('hive-delim-text/carriage-return', StructType([StructField("str", StringType())]), {}), ('hive-delim-text/carriage-return-err', StructType([StructField("str", StringType())]), {}), ], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_basic_hive_text_read(std_input_path, name, schema, spark_tmp_table_factory, options): assert_gpu_and_cpu_are_equal_collect(read_hive_text_sql(std_input_path + '/' + name, schema, spark_tmp_table_factory, options), @@ -240,7 +240,7 @@ def read_hive_text_table(spark, text_table_name, fields="my_field"): "https://github.com/NVIDIA/spark-rapids/pull/7628") @approximate_float @pytest.mark.parametrize('data_gen', hive_text_supported_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hive_text_round_trip(spark_tmp_path, data_gen, spark_tmp_table_factory): gen = StructGen([('my_field', data_gen)], nullable=False) data_path = spark_tmp_path + '/hive_text_table' @@ -282,9 +282,8 @@ def read_hive_text_table_partitions(spark, text_table_name, partition): reason="Hive text reads are disabled on CDH, as per " "https://github.com/NVIDIA/spark-rapids/pull/7628") @approximate_float -@allow_non_gpu("EqualTo,IsNotNull,Literal") # Accounts for partition predicate: `WHERE dt='1'` +@allow_non_gpu("EqualTo,IsNotNull,Literal", *non_utc_allow) # Accounts for partition predicate: `WHERE dt='1'` @pytest.mark.parametrize('data_gen', hive_text_supported_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_hive_text_round_trip_partitioned(spark_tmp_path, data_gen, spark_tmp_table_factory): gen = StructGen([('my_field', data_gen)], nullable=False) data_path = spark_tmp_path + '/hive_text_table' @@ -301,9 +300,8 @@ def test_hive_text_round_trip_partitioned(spark_tmp_path, data_gen, spark_tmp_ta reason="Hive text reads are disabled on CDH, as per " "https://github.com/NVIDIA/spark-rapids/pull/7628") @approximate_float -@allow_non_gpu("EqualTo,IsNotNull,Literal,Or") # Accounts for partition predicate +@allow_non_gpu("EqualTo,IsNotNull,Literal,Or", *non_utc_allow) # Accounts for partition predicate @pytest.mark.parametrize('data_gen', hive_text_supported_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_hive_text_round_trip_two_partitions(spark_tmp_path, data_gen, spark_tmp_table_factory): """ Added to reproduce: https://github.com/NVIDIA/spark-rapids/issues/7383 @@ -529,7 +527,7 @@ def create_table_with_compressed_files(spark): ('hive-delim-text/carriage-return', StructType([StructField("str", StringType())]), {}), ('hive-delim-text/carriage-return-err', StructType([StructField("str", StringType())]), {}), ], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_basic_hive_text_write(std_input_path, input_dir, schema, spark_tmp_table_factory, mode, options): # Configure table options, including schema. if options is None: diff --git a/integration_tests/src/main/python/hive_write_test.py b/integration_tests/src/main/python/hive_write_test.py index 7bc5ceede85..ae7052dffd7 100644 --- a/integration_tests/src/main/python/hive_write_test.py +++ b/integration_tests/src/main/python/hive_write_test.py @@ -59,7 +59,7 @@ def _restricted_timestamp(nullable=True): @pytest.mark.skipif(not is_hive_available(), reason="Hive is missing") @pytest.mark.parametrize("gens", _write_gens, ids=idfn) @pytest.mark.parametrize("storage", ["PARQUET", "nativeorc", "hiveorc"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_optimized_hive_ctas_basic(gens, storage, spark_tmp_table_factory): data_table = spark_tmp_table_factory.get() gen_list = [('c' + str(i), gen) for i, gen in enumerate(gens)] diff --git a/integration_tests/src/main/python/join_test.py b/integration_tests/src/main/python/join_test.py index 9ea78a85401..6660e663c92 100644 --- a/integration_tests/src/main/python/join_test.py +++ b/integration_tests/src/main/python/join_test.py @@ -189,11 +189,11 @@ def do_join(spark): # For floating point values the normalization is done using a higher order function. We could probably work around this # for now it falls back to the CPU @allow_non_gpu('SortMergeJoinExec', 'SortExec', 'ArrayTransform', 'LambdaFunction', - 'NamedLambdaVariable', 'NormalizeNaNAndZero', 'ShuffleExchangeExec', 'HashPartitioning') + 'NamedLambdaVariable', 'NormalizeNaNAndZero', 'ShuffleExchangeExec', 'HashPartitioning', + *non_utc_allow) @ignore_order(local=True) @pytest.mark.parametrize('data_gen', single_level_array_gens + [binary_gen], ids=idfn) @pytest.mark.parametrize('join_type', all_join_types, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_sortmerge_join_wrong_key_fallback(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 500, 500) @@ -213,7 +213,7 @@ def do_join(spark): @pytest.mark.parametrize('data_gen', basic_nested_gens + [decimal_gen_128bit], ids=idfn) @pytest.mark.parametrize('join_type', all_join_types, ids=idfn) @pytest.mark.parametrize('sub_part_enabled', ['false', 'true'], ids=['SubPartition_OFF', 'SubPartition_ON']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_join_ridealong(data_gen, join_type, sub_part_enabled): def do_join(spark): left, right = create_ridealong_df(spark, short_gen, data_gen, 50, 500) @@ -230,7 +230,7 @@ def do_join(spark): # Not all join types can be translated to a broadcast join, but this tests them to be sure we # can handle what spark is doing @pytest.mark.parametrize('join_type', all_join_types, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_broadcast_join_right_table(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 500, 250) @@ -242,7 +242,7 @@ def do_join(spark): # Not all join types can be translated to a broadcast join, but this tests them to be sure we # can handle what spark is doing @pytest.mark.parametrize('join_type', all_join_types, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_broadcast_join_right_table_ridealong(data_gen, join_type): def do_join(spark): left, right = create_ridealong_df(spark, short_gen, data_gen, 500, 500) @@ -256,7 +256,7 @@ def do_join(spark): # Not all join types can be translated to a broadcast join, but this tests them to be sure we # can handle what spark is doing @pytest.mark.parametrize('join_type', all_join_types, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_broadcast_join_right_table_with_job_group(data_gen, join_type): with_cpu_session(lambda spark : spark.sparkContext.setJobGroup("testjob1", "test", False)) def do_join(spark): @@ -271,7 +271,7 @@ def do_join(spark): @pytest.mark.parametrize('data_gen,batch_size', join_batch_size_test_params( (all_gen + basic_nested_gens, '1g'), (join_small_batch_gens + [basic_struct_gen, ArrayGen(string_gen)], '100')), ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cartesian_join(data_gen, batch_size): def do_join(spark): left, right = create_df(spark, data_gen, 50, 25) @@ -311,7 +311,7 @@ def do_join(spark): @pytest.mark.parametrize('data_gen,batch_size', join_batch_size_test_params( (all_gen, '1g'), (join_small_batch_gens, '100')), ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_cartesian_join_with_condition(data_gen, batch_size): def do_join(spark): left, right = create_df(spark, data_gen, 50, 25) @@ -329,7 +329,7 @@ def do_join(spark): @pytest.mark.parametrize('data_gen,batch_size', join_batch_size_test_params( (all_gen + basic_nested_gens, '1g'), (join_small_batch_gens, '100')), ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_broadcast_nested_loop_join(data_gen, batch_size): def do_join(spark): left, right = create_df(spark, data_gen, 50, 25) @@ -365,7 +365,7 @@ def do_join(spark): (join_ast_gen, '1g'), ([int_gen], 100)), ids=idfn) @pytest.mark.parametrize('join_type', ['Left', 'Inner', 'LeftSemi', 'LeftAnti', 'Cross'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_right_broadcast_nested_loop_join_with_ast_condition(data_gen, join_type, batch_size): def do_join(spark): left, right = create_df(spark, data_gen, 50, 25) @@ -380,7 +380,7 @@ def do_join(spark): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.parametrize('data_gen', join_ast_gen, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_left_broadcast_nested_loop_join_with_ast_condition(data_gen): def do_join(spark): left, right = create_df(spark, data_gen, 50, 25) @@ -439,7 +439,7 @@ def do_join(spark): float_gen, double_gen, string_gen, boolean_gen, date_gen, timestamp_gen], ids=idfn) @pytest.mark.parametrize('join_type', ['Left', 'Right', 'FullOuter', 'LeftSemi', 'LeftAnti'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_broadcast_nested_loop_join_with_array_contains(data_gen, join_type): arr_gen = ArrayGen(data_gen) literal = with_cpu_session(lambda spark: gen_scalar(data_gen)) @@ -452,7 +452,7 @@ def do_join(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', all_gen, ids=idfn) @pytest.mark.parametrize('join_type', ['Left', 'LeftSemi', 'LeftAnti'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_right_broadcast_nested_loop_join_condition_missing(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 50, 25) @@ -468,7 +468,7 @@ def do_join(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', all_gen, ids=idfn) @pytest.mark.parametrize('join_type', ['Right'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_left_broadcast_nested_loop_join_condition_missing(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 50, 25) @@ -483,7 +483,7 @@ def do_join(spark): @pytest.mark.parametrize('data_gen', all_gen + single_level_array_gens + [binary_gen], ids=idfn) @pytest.mark.parametrize('join_type', ['Left', 'LeftSemi', 'LeftAnti'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_right_broadcast_nested_loop_join_condition_missing_count(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 50, 25) @@ -492,29 +492,27 @@ def do_join(spark): @pytest.mark.parametrize('data_gen', all_gen + single_level_array_gens + [binary_gen], ids=idfn) @pytest.mark.parametrize('join_type', ['Right'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_left_broadcast_nested_loop_join_condition_missing_count(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 50, 25) return broadcast(left).join(right, how=join_type).selectExpr('COUNT(*)') assert_gpu_and_cpu_are_equal_collect(do_join) -@allow_non_gpu('BroadcastExchangeExec', 'BroadcastNestedLoopJoinExec', 'GreaterThanOrEqual') +@allow_non_gpu('BroadcastExchangeExec', 'BroadcastNestedLoopJoinExec', 'GreaterThanOrEqual', *non_utc_allow) @ignore_order(local=True) @pytest.mark.parametrize('data_gen', all_gen, ids=idfn) @pytest.mark.parametrize('join_type', ['LeftOuter', 'LeftSemi', 'LeftAnti', 'FullOuter'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_broadcast_nested_loop_join_with_conditionals_build_left_fallback(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 50, 25) return broadcast(left).join(right, (left.b >= right.r_b), join_type) assert_gpu_fallback_collect(do_join, 'BroadcastNestedLoopJoinExec') -@allow_non_gpu('BroadcastExchangeExec', 'BroadcastNestedLoopJoinExec', 'GreaterThanOrEqual') +@allow_non_gpu('BroadcastExchangeExec', 'BroadcastNestedLoopJoinExec', 'GreaterThanOrEqual', *non_utc_allow) @ignore_order(local=True) @pytest.mark.parametrize('data_gen', all_gen, ids=idfn) @pytest.mark.parametrize('join_type', ['RightOuter', 'FullOuter'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_broadcast_nested_loop_with_conditionals_build_right_fallback(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 50, 25) @@ -531,7 +529,7 @@ def do_join(spark): # Specify 200 shuffle partitions to test cases where streaming side is empty # as in https://github.com/NVIDIA/spark-rapids/issues/7516 @pytest.mark.parametrize('shuffle_conf', [{}, {'spark.sql.shuffle.partitions': 200}], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_broadcast_join_left_table(data_gen, join_type, shuffle_conf): def do_join(spark): left, right = create_df(spark, data_gen, 250, 500) @@ -543,7 +541,7 @@ def do_join(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', join_ast_gen, ids=idfn) @pytest.mark.parametrize('join_type', all_join_types, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_broadcast_join_with_conditionals(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 500, 250) @@ -598,7 +596,7 @@ def do_join(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', join_ast_gen, ids=idfn) @pytest.mark.parametrize('join_type', ['Left', 'Right', 'Inner', 'FullOuter', 'LeftSemi', 'LeftAnti'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_sortmerge_join_with_condition_ast(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 500, 250) @@ -715,7 +713,7 @@ def do_join(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', struct_gens, ids=idfn) @pytest.mark.parametrize('join_type', ['Inner', 'Left', 'Right', 'Cross', 'LeftSemi', 'LeftAnti'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_sortmerge_join_struct_as_key(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 500, 250) @@ -727,7 +725,7 @@ def do_join(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', struct_gens, ids=idfn) @pytest.mark.parametrize('join_type', ['Inner', 'Left', 'Right', 'Cross', 'LeftSemi', 'LeftAnti'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_sortmerge_join_struct_mixed_key(data_gen, join_type): def do_join(spark): left = two_col_df(spark, data_gen, int_gen, length=500) @@ -740,7 +738,7 @@ def do_join(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', struct_gens, ids=idfn) @pytest.mark.parametrize('join_type', ['Inner', 'Left', 'Right', 'Cross', 'LeftSemi', 'LeftAnti'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_sortmerge_join_struct_mixed_key_with_null_filter(data_gen, join_type): def do_join(spark): left = two_col_df(spark, data_gen, int_gen, length=500) @@ -755,7 +753,7 @@ def do_join(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', struct_gens, ids=idfn) @pytest.mark.parametrize('join_type', ['Inner', 'Left', 'Right', 'Cross', 'LeftSemi', 'LeftAnti'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_broadcast_join_right_struct_as_key(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 500, 250) @@ -767,7 +765,7 @@ def do_join(spark): @ignore_order(local=True) @pytest.mark.parametrize('data_gen', struct_gens, ids=idfn) @pytest.mark.parametrize('join_type', ['Inner', 'Left', 'Right', 'Cross', 'LeftSemi', 'LeftAnti'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_broadcast_join_right_struct_mixed_key(data_gen, join_type): def do_join(spark): left = two_col_df(spark, data_gen, int_gen, length=500) @@ -788,11 +786,11 @@ def do_join(spark): assert_gpu_and_cpu_are_equal_collect(do_join, conf=_sortmerge_join_conf) @allow_non_gpu('SortMergeJoinExec', 'SortExec', 'NormalizeNaNAndZero', 'CreateNamedStruct', - 'GetStructField', 'Literal', 'If', 'IsNull', 'ShuffleExchangeExec', 'HashPartitioning') + 'GetStructField', 'Literal', 'If', 'IsNull', 'ShuffleExchangeExec', 'HashPartitioning', + *non_utc_allow) @ignore_order(local=True) @pytest.mark.parametrize('data_gen', struct_gens, ids=idfn) @pytest.mark.parametrize('join_type', ['FullOuter'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_sortmerge_join_struct_as_key_fallback(data_gen, join_type): def do_join(spark): left, right = create_df(spark, data_gen, 500, 500) diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 16e28b724c2..2b4161b3dec 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -205,21 +205,20 @@ def test_json_ts_formats_round_trip(spark_tmp_path, date_format, ts_part, v1_ena .json(data_path), conf=updated_conf) -@allow_non_gpu('FileSourceScanExec', 'ProjectExec') +@allow_non_gpu('FileSourceScanExec', 'ProjectExec', *non_utc_allow) @pytest.mark.skipif(is_before_spark_341(), reason='`TIMESTAMP_NTZ` is only supported in PySpark 341+') @pytest.mark.parametrize('ts_part', json_supported_ts_parts) @pytest.mark.parametrize('date_format', json_supported_date_formats) @pytest.mark.parametrize("timestamp_type", ["TIMESTAMP_LTZ", "TIMESTAMP_NTZ"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_json_ts_formats_round_trip_ntz_v1(spark_tmp_path, date_format, ts_part, timestamp_type): json_ts_formats_round_trip_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, 'json', 'FileSourceScanExec') -@allow_non_gpu('BatchScanExec', 'ProjectExec') +@allow_non_gpu('BatchScanExec', 'ProjectExec', *non_utc_allow) @pytest.mark.skipif(is_before_spark_341(), reason='`TIMESTAMP_NTZ` is only supported in PySpark 341+') @pytest.mark.parametrize('ts_part', json_supported_ts_parts) @pytest.mark.parametrize('date_format', json_supported_date_formats) @pytest.mark.parametrize("timestamp_type", ["TIMESTAMP_LTZ", "TIMESTAMP_NTZ"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') + def test_json_ts_formats_round_trip_ntz_v2(spark_tmp_path, date_format, ts_part, timestamp_type): json_ts_formats_round_trip_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, '', 'BatchScanExec') @@ -399,7 +398,7 @@ def test_json_read_invalid_dates(std_input_path, filename, schema, read_func, an 'CORRECTED', 'EXCEPTION' ]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_json_read_valid_timestamps(std_input_path, filename, schema, read_func, ansi_enabled, time_parser_policy, \ spark_tmp_table_factory): updated_conf = copy_and_update(_enable_all_types_conf, @@ -457,7 +456,7 @@ def test_json_read_count(spark_tmp_path, v1_enabled_list): lambda spark : spark.read.schema(schema).json(data_path), conf=updated_conf) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_from_json_map(): # The test here is working around some inconsistencies in how the keys are parsed for maps # on the GPU the keys are dense, but on the CPU they are sparse @@ -492,7 +491,7 @@ def test_from_json_map_fallback(): 'struct', 'struct', ]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_from_json_struct(schema): # note that column 'a' does not use leading zeroes due to https://github.com/NVIDIA/spark-rapids/issues/9588 json_string_gen = StringGen(r'{"a": [1-9]{0,5}, "b": "[A-Z]{0,5}", "c": 1\d\d\d}') \ @@ -512,7 +511,7 @@ def test_from_json_struct(schema): r'{ "bool": [0-9]{4}-[0-9]{2}-[0-9]{2} }', r'{ "bool": "[0-9]{4}-[0-9]{2}-[0-9]{2}" }' ]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_from_json_struct_boolean(pattern): json_string_gen = StringGen(pattern) \ .with_special_case('', weight=50) \ @@ -522,7 +521,7 @@ def test_from_json_struct_boolean(pattern): .select(f.col('a'), f.from_json('a', 'struct')), conf={"spark.rapids.sql.expression.JsonToStructs": True}) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_from_json_struct_decimal(): json_string_gen = StringGen(r'{ "a": "[+-]?([0-9]{0,5})?(\.[0-9]{0,2})?([eE][+-]?[0-9]{1,2})?" }') \ .with_special_pattern('', weight=50) \ @@ -562,7 +561,7 @@ def test_from_json_struct_decimal(): pytest.param("LEGACY", marks=pytest.mark.allow_non_gpu('ProjectExec')), "CORRECTED" ]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_from_json_struct_date(date_gen, date_format, time_parser_policy): json_string_gen = StringGen(r'{ "a": ' + date_gen + ' }') \ .with_special_case('{ "a": null }') \ @@ -645,7 +644,7 @@ def test_from_json_struct_date_fallback_non_default_format(date_gen, date_format "CORRECTED" ]) @pytest.mark.parametrize('ansi_enabled', [ True, False ]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_from_json_struct_timestamp(timestamp_gen, timestamp_format, time_parser_policy, ansi_enabled): json_string_gen = StringGen(r'{ "a": ' + timestamp_gen + ' }') \ .with_special_case('{ "a": null }') \ @@ -697,7 +696,7 @@ def test_from_json_struct_timestamp_fallback_non_default_format(timestamp_gen, t @pytest.mark.parametrize('schema', ['struct', 'struct>', 'struct>']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_from_json_struct_of_struct(schema): json_string_gen = StringGen(r'{"teacher": "[A-Z]{1}[a-z]{2,5}",' \ r'"student": {"name": "[A-Z]{1}[a-z]{2,5}", "age": 1\d}}') \ @@ -712,7 +711,7 @@ def test_from_json_struct_of_struct(schema): @pytest.mark.parametrize('schema', ['struct', 'struct>>', 'struct>>']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_from_json_struct_of_list(schema): json_string_gen = StringGen(r'{"teacher": "[A-Z]{1}[a-z]{2,5}",' \ r'"student": \[{"name": "[A-Z]{1}[a-z]{2,5}", "class": "junior"},' \ @@ -725,7 +724,7 @@ def test_from_json_struct_of_list(schema): conf={"spark.rapids.sql.expression.JsonToStructs": True}) @pytest.mark.parametrize('schema', ['struct', 'struct']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_from_json_struct_all_empty_string_input(schema): json_string_gen = StringGen('') assert_gpu_and_cpu_are_equal_collect( @@ -802,7 +801,7 @@ def test_read_case_col_name(spark_tmp_path, v1_enabled_list, col_name): pytest.param(True, marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/9517')), False ]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_structs_to_json(spark_tmp_path, data_gen, ignore_null_fields, pretty): struct_gen = StructGen([ ('a', data_gen), diff --git a/integration_tests/src/main/python/map_test.py b/integration_tests/src/main/python/map_test.py index dbd02884ddc..d3f332dce7f 100644 --- a/integration_tests/src/main/python/map_test.py +++ b/integration_tests/src/main/python/map_test.py @@ -57,7 +57,7 @@ @pytest.mark.parametrize('data_gen', supported_key_map_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_map_keys(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -69,7 +69,7 @@ def test_map_keys(data_gen): @pytest.mark.parametrize('data_gen', supported_key_map_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_map_values(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -152,7 +152,7 @@ def test_get_map_value_numeric_keys(data_gen): @pytest.mark.parametrize('data_gen', supported_key_map_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_get_map_value_supported_keys(data_gen): key_gen = data_gen._key_gen # first expression is not guaranteed to hit @@ -191,7 +191,7 @@ def query_map_scalar(spark): @allow_non_gpu('WindowLocalExec') @datagen_overrides(seed=0, reason='https://github.com/NVIDIA/spark-rapids/issues/9683') @pytest.mark.parametrize('data_gen', supported_key_map_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_map_scalars_supported_key_types(data_gen): key_gen = data_gen._key_gen def query_map_scalar(spark): @@ -229,7 +229,7 @@ def query_map_scalar(spark): @pytest.mark.parametrize('data_gen', [MapGen(DateGen(nullable=False), value(), max_length=6) for value in get_map_value_gens()], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_get_map_value_date_keys(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -241,7 +241,7 @@ def test_get_map_value_date_keys(data_gen): @pytest.mark.parametrize('data_gen', [MapGen(TimestampGen(nullable=False), value(), max_length=6) for value in get_map_value_gens()], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_get_map_value_timestamp_keys(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -260,7 +260,7 @@ def test_map_side_effects(): @pytest.mark.parametrize('key_gen', [StringGen(nullable=False), IntegerGen(nullable=False), basic_struct_gen], ids=idfn) @pytest.mark.parametrize('value_gen', [StringGen(nullable=True), IntegerGen(nullable=True), basic_struct_gen], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_single_entry_map(key_gen, value_gen): data_gen = [('a', key_gen), ('b', value_gen)] assert_gpu_and_cpu_are_equal_collect( @@ -464,7 +464,7 @@ def test_simple_get_map_value_with_strict_index(strict_index, data_gen): [MapGen(StringGen(pattern='key_[0-9]', nullable=False), value(), max_length=6) for value in get_map_value_gens()], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_element_at_map_string_keys(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -478,7 +478,7 @@ def test_element_at_map_string_keys(data_gen): @pytest.mark.parametrize('data_gen', numeric_key_map_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_element_at_map_numeric_keys(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -494,7 +494,7 @@ def test_element_at_map_numeric_keys(data_gen): [MapGen(DecimalGen(precision=35, scale=2, nullable=False), value(), max_length=6) for value in get_map_value_gens(precision=37, scale=0)], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_get_map_value_element_at_map_dec_col_keys(data_gen): keys = DecimalGen(precision=35, scale=2) assert_gpu_and_cpu_are_equal_collect( @@ -520,7 +520,7 @@ def test_get_map_value_element_at_map_string_col_keys_ansi(data_gen, ansi): [MapGen(StringGen(pattern='key_[0-9]', nullable=False), value(), max_length=6) for value in get_map_value_gens(precision=37, scale=0)], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_get_map_value_element_at_map_string_col_keys(data_gen): keys = StringGen(pattern='key_[0-9]') assert_gpu_and_cpu_are_equal_collect( @@ -577,7 +577,7 @@ def test_get_map_value_string_col_keys_ansi_null(data_gen): @pytest.mark.parametrize('data_gen', [MapGen(DateGen(nullable=False), value(), max_length=6) for value in get_map_value_gens()], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_element_at_map_date_keys(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -591,7 +591,7 @@ def test_element_at_map_date_keys(data_gen): [MapGen(TimestampGen(nullable=False), value(), max_length=6) for value in get_map_value_gens()], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_element_at_map_timestamp_keys(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -623,7 +623,7 @@ def test_map_element_at_ansi_null(data_gen): conf=ansi_enabled_conf) @pytest.mark.parametrize('data_gen', map_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_transform_values(data_gen): def do_it(spark): columns = ['a', 'b', @@ -662,7 +662,7 @@ def do_it(spark): @pytest.mark.parametrize('data_gen', map_gens_sample + decimal_128_map_gens + decimal_64_map_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_transform_keys(data_gen): # The processing here is very limited, because we need to be sure we do not create duplicate keys. # This can happen because of integer overflow, round off errors in floating point, etc. So for now @@ -722,7 +722,7 @@ def test_sql_map_scalars(query): @pytest.mark.parametrize('data_gen', map_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_map_filter(data_gen): columns = ['map_filter(a, (key, value) -> isnotnull(value) )', 'map_filter(a, (key, value) -> isnull(value) )', diff --git a/integration_tests/src/main/python/orc_cast_test.py b/integration_tests/src/main/python/orc_cast_test.py index cccd60125b9..5f6838df688 100644 --- a/integration_tests/src/main/python/orc_cast_test.py +++ b/integration_tests/src/main/python/orc_cast_test.py @@ -17,6 +17,7 @@ from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error from conftest import is_not_utc from data_gen import * +from marks import allow_non_gpu from pyspark.sql.types import * from spark_session import with_cpu_session from orc_test import reader_opt_confs @@ -50,7 +51,7 @@ def test_casting_among_integer_types(spark_tmp_path, reader_confs, v1_enabled_li @pytest.mark.parametrize('to_type', ['float', 'double', 'string', 'timestamp']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_casting_from_integer(spark_tmp_path, to_type): orc_path = spark_tmp_path + '/orc_cast_integer' # The Python 'datetime' module only supports a max-year of 10000, so we set the Long type max @@ -72,7 +73,7 @@ def test_casting_from_integer(spark_tmp_path, to_type): @pytest.mark.parametrize('overflow_long_gen', [LongGen(min_val=int(1e16)), LongGen(max_val=int(-1e16))]) @pytest.mark.parametrize('to_type', ['timestamp']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_casting_from_overflow_long(spark_tmp_path, overflow_long_gen,to_type): # Timestamp(micro-seconds) is actually type of int64, when casting long(int64) to timestamp, # we need to multiply 1e6 (or 1e3), and it may cause overflow. This function aims to test @@ -103,7 +104,7 @@ def test_casting_from_float_and_double(spark_tmp_path, to_type): @pytest.mark.parametrize('data_gen', [DoubleGen(max_exp=32, special_cases=None), DoubleGen(max_exp=32, special_cases=[8.88e9, 9.99e10, 1.314e11])]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_casting_from_double_to_timestamp(spark_tmp_path, data_gen): # ORC will assume the original double value in seconds, we need to convert them to # timestamp(INT64 in micro-seconds). @@ -127,7 +128,7 @@ def test_casting_from_double_to_timestamp(spark_tmp_path, data_gen): ) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_casting_from_overflow_double_to_timestamp(spark_tmp_path): orc_path = spark_tmp_path + '/orc_casting_from_overflow_double_to_timestamp' with_cpu_session( diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index 409d0850987..f539e5f6c74 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -68,7 +68,7 @@ def get_orc_timestamp_gen(nullable=True): @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('orc_impl', ["native", "hive"]) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_basic_read(std_input_path, name, read_func, v1_enabled_list, orc_impl, reader_confs): all_confs = copy_and_update(reader_confs, { 'spark.sql.sources.useV1SourceList': v1_enabled_list, @@ -160,7 +160,7 @@ def test_orc_fallback(spark_tmp_path, read_func, disable_conf): @pytest.mark.parametrize('read_func', [read_orc_df, read_orc_sql]) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_read_round_trip(spark_tmp_path, orc_gens, read_func, reader_confs, v1_enabled_list): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)] data_path = spark_tmp_path + '/ORC_DATA' @@ -186,7 +186,7 @@ def test_read_round_trip(spark_tmp_path, orc_gens, read_func, reader_confs, v1_e @pytest.mark.parametrize('read_func', [read_orc_df, read_orc_sql]) @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_pred_push_round_trip(spark_tmp_path, orc_gen, read_func, v1_enabled_list, reader_confs): data_path = spark_tmp_path + '/ORC_DATA' # Append two struct columns to verify nested predicate pushdown. @@ -243,7 +243,7 @@ def test_compress_read_round_trip(spark_tmp_path, compress, v1_enabled_list, rea @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_simple_partitioned_read(spark_tmp_path, v1_enabled_list, reader_confs): # Once https://github.com/NVIDIA/spark-rapids/issues/131 is fixed # we should go with a more standard set of generators @@ -310,7 +310,7 @@ def test_partitioned_read_just_partitions(spark_tmp_path, v1_enabled_list, reade @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_merge_schema_read(spark_tmp_path, v1_enabled_list, reader_confs): # Once https://github.com/NVIDIA/spark-rapids/issues/131 is fixed # we should go with a more standard set of generators @@ -589,7 +589,7 @@ def test_read_struct_without_stream(spark_tmp_path): @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('case_sensitive', ["false", "true"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_read_with_more_columns(spark_tmp_path, orc_gen, reader_confs, v1_enabled_list, case_sensitive): struct_gen = StructGen([('nested_col', orc_gen)]) # Map is not supported yet. @@ -777,7 +777,7 @@ def test_orc_read_varchar_as_string(std_input_path): @pytest.mark.parametrize('gens', orc_gens_list, ids=idfn) @pytest.mark.parametrize('keep_order', [True, pytest.param(False, marks=pytest.mark.ignore_order(local=True))]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_read_round_trip_for_multithreaded_combining(spark_tmp_path, gens, keep_order): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(gens)] data_path = spark_tmp_path + '/ORC_DATA' @@ -792,7 +792,7 @@ def test_read_round_trip_for_multithreaded_combining(spark_tmp_path, gens, keep_ @pytest.mark.parametrize('keep_order', [True, pytest.param(False, marks=pytest.mark.ignore_order(local=True))]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_simple_partitioned_read_for_multithreaded_combining(spark_tmp_path, keep_order): orc_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), diff --git a/integration_tests/src/main/python/orc_write_test.py b/integration_tests/src/main/python/orc_write_test.py index 5617f8e20e5..c6244921677 100644 --- a/integration_tests/src/main/python/orc_write_test.py +++ b/integration_tests/src/main/python/orc_write_test.py @@ -81,7 +81,7 @@ @pytest.mark.parametrize('orc_gens', orc_write_gens_list, ids=idfn) @pytest.mark.parametrize('orc_impl', ["native", "hive"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_write_round_trip(spark_tmp_path, orc_gens, orc_impl): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)] data_path = spark_tmp_path + '/ORC_DATA' @@ -116,7 +116,7 @@ def test_write_round_trip_corner(spark_tmp_path, orc_gen, orc_impl): # There are race conditions around when individual files are read in for partitioned data @ignore_order @pytest.mark.parametrize('orc_gen', orc_part_write_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_part_write_round_trip(spark_tmp_path, orc_gen): gen_list = [('a', RepeatSeqGen(orc_gen, 10)), ('b', orc_gen)] @@ -170,8 +170,8 @@ def test_compress_write_round_trip(spark_tmp_path, compress): @pytest.mark.order(2) @pytest.mark.parametrize('orc_gens', orc_write_gens_list, ids=idfn) @pytest.mark.parametrize('orc_impl', ["native", "hive"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') -def test_write_save_table(spark_tmp_path, orc_gens, orc_impl, spark_tmp_table_factory): +@allow_non_gpu(*non_utc_allow) +def test_write_save_table_orc(spark_tmp_path, orc_gens, orc_impl, spark_tmp_table_factory): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)] data_path = spark_tmp_path + '/ORC_DATA' all_confs={'spark.sql.sources.useV1SourceList': "orc", @@ -193,7 +193,7 @@ def write_orc_sql_from(spark, df, data_path, write_to_table): @pytest.mark.parametrize('orc_gens', orc_write_gens_list, ids=idfn) @pytest.mark.parametrize('ts_type', ["TIMESTAMP_MICROS", "TIMESTAMP_MILLIS"]) @pytest.mark.parametrize('orc_impl', ["native", "hive"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_write_sql_save_table(spark_tmp_path, orc_gens, ts_type, orc_impl, spark_tmp_table_factory): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)] data_path = spark_tmp_path + '/ORC_DATA' @@ -203,9 +203,8 @@ def test_write_sql_save_table(spark_tmp_path, orc_gens, ts_type, orc_impl, spark data_path, conf={'spark.sql.orc.impl': orc_impl, 'spark.rapids.sql.format.orc.write.enabled': True}) -@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec') +@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec', *non_utc_allow) @pytest.mark.parametrize('codec', ['zlib', 'lzo']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_orc_write_compression_fallback(spark_tmp_path, codec, spark_tmp_table_factory): gen = TimestampGen() data_path = spark_tmp_path + '/PARQUET_DATA' @@ -262,7 +261,7 @@ def sql_write(spark, path): @pytest.mark.parametrize('orc_gens', orc_write_gens_list, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_write_empty_orc_round_trip(spark_tmp_path, orc_gens): def create_empty_df(spark, path): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)] diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py index dd29be6bd3b..5d490f71029 100644 --- a/integration_tests/src/main/python/parquet_test.py +++ b/integration_tests/src/main/python/parquet_test.py @@ -164,7 +164,7 @@ def setup_table(spark): @pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) @pytest.mark.parametrize('reader_confs', reader_opt_confs) @pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_parquet_read_round_trip(spark_tmp_path, parquet_gens, read_func, reader_confs, v1_enabled_list): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] data_path = spark_tmp_path + '/PARQUET_DATA' @@ -299,7 +299,7 @@ def test_parquet_compress_read_round_trip(spark_tmp_path, compress, v1_enabled_l @pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) @pytest.mark.parametrize('reader_confs', reader_opt_confs) @pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_parquet_pred_push_round_trip(spark_tmp_path, parquet_gen, read_func, v1_enabled_list, reader_confs): data_path = spark_tmp_path + '/PARQUET_DATA' gen_list = [('a', RepeatSeqGen(parquet_gen, 100)), ('b', parquet_gen)] @@ -559,7 +559,7 @@ def test_parquet_read_merge_schema(spark_tmp_path, v1_enabled_list, reader_confs @pytest.mark.parametrize('reader_confs', reader_opt_confs) @pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_parquet_read_merge_schema_from_conf(spark_tmp_path, v1_enabled_list, reader_confs): # Once https://github.com/NVIDIA/spark-rapids/issues/133 and https://github.com/NVIDIA/spark-rapids/issues/132 are fixed # we should go with a more standard set of generators @@ -875,7 +875,7 @@ def test_parquet_reading_from_unaligned_pages_basic_filters(spark_tmp_path, read @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) @pytest.mark.parametrize('enable_dictionary', ["true", "false"], ids=idfn) @pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_parquet_reading_from_unaligned_pages_all_types(spark_tmp_path, reader_confs, enable_dictionary, v1_enabled_list): all_confs = copy_and_update(reader_confs, {'spark.sql.sources.useV1SourceList': v1_enabled_list}) data_path = spark_tmp_path + '/PARQUET_UNALIGNED_DATA' @@ -903,7 +903,7 @@ def test_parquet_reading_from_unaligned_pages_all_types(spark_tmp_path, reader_c @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) @pytest.mark.parametrize('enable_dictionary', ["true", "false"], ids=idfn) @pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_parquet_reading_from_unaligned_pages_all_types_dict_optimized(spark_tmp_path, reader_confs, enable_dictionary, v1_enabled_list): all_confs = copy_and_update(reader_confs, {'spark.sql.sources.useV1SourceList': v1_enabled_list}) data_path = spark_tmp_path + '/PARQUET_UNALIGNED_DATA' diff --git a/integration_tests/src/main/python/parquet_testing_test.py b/integration_tests/src/main/python/parquet_testing_test.py index a4600de7b86..abf06d11e5f 100644 --- a/integration_tests/src/main/python/parquet_testing_test.py +++ b/integration_tests/src/main/python/parquet_testing_test.py @@ -17,7 +17,8 @@ from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error from conftest import get_std_input_path, is_parquet_testing_tests_forced, is_precommit_run, is_not_utc -from data_gen import copy_and_update +from data_gen import * +from marks import allow_non_gpu from pathlib import Path import pytest from spark_session import is_before_spark_330, is_spark_350_or_later @@ -122,7 +123,7 @@ def gen_testing_params_for_valid_files(): @pytest.mark.parametrize("path", gen_testing_params_for_valid_files()) @pytest.mark.parametrize("confs", [_native_reader_confs, _java_reader_confs]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_parquet_testing_valid_files(path, confs): assert_gpu_and_cpu_are_equal_collect(lambda spark: spark.read.parquet(path), conf=confs) diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index 9584f2a3520..00079fffe0d 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -90,7 +90,7 @@ @pytest.mark.order(1) # at the head of xdist worker queue if pytest-order is installed @pytest.mark.parametrize('parquet_gens', parquet_write_gens_list, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_write_round_trip(spark_tmp_path, parquet_gens): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] data_path = spark_tmp_path + '/PARQUET_DATA' @@ -136,7 +136,7 @@ def test_write_round_trip_corner(spark_tmp_path, par_gen): ArrayGen(TimestampGen(), max_length=10), MapGen(TimestampGen(nullable=False), TimestampGen())]], ids=idfn) @pytest.mark.parametrize('ts_type', parquet_ts_write_options) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_timestamp_write_round_trip(spark_tmp_path, parquet_gens, ts_type): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] data_path = spark_tmp_path + '/PARQUET_DATA' @@ -150,7 +150,7 @@ def test_timestamp_write_round_trip(spark_tmp_path, parquet_gens, ts_type): @pytest.mark.parametrize('ts_type', parquet_ts_write_options) @pytest.mark.parametrize('ts_rebase', ['CORRECTED']) @ignore_order -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_write_ts_millis(spark_tmp_path, ts_type, ts_rebase): gen = TimestampGen() data_path = spark_tmp_path + '/PARQUET_DATA' @@ -174,7 +174,7 @@ def test_write_ts_millis(spark_tmp_path, ts_type, ts_rebase): @ignore_order @pytest.mark.order(1) # at the head of xdist worker queue if pytest-order is installed @pytest.mark.parametrize('parquet_gen', parquet_part_write_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_part_write_round_trip(spark_tmp_path, parquet_gen): gen_list = [('a', RepeatSeqGen(parquet_gen, 10)), ('b', parquet_gen)] @@ -188,8 +188,7 @@ def test_part_write_round_trip(spark_tmp_path, parquet_gen): @pytest.mark.skipif(is_spark_340_or_later() or is_databricks122_or_later(), reason="`WriteFilesExec` is only supported in Spark 340+") @pytest.mark.parametrize('data_gen', [TimestampGen()], ids=idfn) -@pytest.mark.allow_non_gpu("DataWritingCommandExec") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@pytest.mark.allow_non_gpu("DataWritingCommandExec", *non_utc_allow) def test_int96_write_conf(spark_tmp_path, data_gen): data_path = spark_tmp_path + '/PARQUET_DATA' confs = copy_and_update(writer_confs, { @@ -206,8 +205,7 @@ def test_int96_write_conf(spark_tmp_path, data_gen): @pytest.mark.skipif(is_before_spark_340() and not is_databricks122_or_later(), reason="`WriteFilesExec` is only supported in Spark 340+") @pytest.mark.parametrize('data_gen', [TimestampGen()], ids=idfn) # Note: From Spark 340, WriteFilesExec is introduced. -@pytest.mark.allow_non_gpu("DataWritingCommandExec", "WriteFilesExec") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@pytest.mark.allow_non_gpu("DataWritingCommandExec", "WriteFilesExec", *non_utc_allow) def test_int96_write_conf_with_write_exec(spark_tmp_path, data_gen): data_path = spark_tmp_path + '/PARQUET_DATA' confs = copy_and_update(writer_confs, { @@ -221,7 +219,7 @@ def test_int96_write_conf_with_write_exec(spark_tmp_path, data_gen): ['DataWritingCommandExec', 'WriteFilesExec'], confs) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_all_null_int96(spark_tmp_path): class AllNullTimestampGen(TimestampGen): def start(self, rand): @@ -251,7 +249,7 @@ def test_compress_write_round_trip(spark_tmp_path, compress): @pytest.mark.order(2) @pytest.mark.parametrize('parquet_gens', parquet_write_gens_list, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_write_save_table(spark_tmp_path, parquet_gens, spark_tmp_table_factory): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] data_path = spark_tmp_path + '/PARQUET_DATA' @@ -269,7 +267,7 @@ def write_parquet_sql_from(spark, df, data_path, write_to_table): @pytest.mark.order(2) @pytest.mark.parametrize('parquet_gens', parquet_write_gens_list, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_write_sql_save_table(spark_tmp_path, parquet_gens, spark_tmp_table_factory): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] data_path = spark_tmp_path + '/PARQUET_DATA' @@ -292,7 +290,7 @@ def writeParquetUpgradeCatchException(spark, df, data_path, spark_tmp_table_fact ('TIMESTAMP_MICROS', TimestampGen(start=datetime(1, 2, 1, tzinfo=timezone.utc), end=datetime(1899, 12, 31, tzinfo=timezone.utc))), ('TIMESTAMP_MILLIS', TimestampGen(start=datetime(1, 2, 1, tzinfo=timezone.utc), end=datetime(1899, 12, 31, tzinfo=timezone.utc)))]) @pytest.mark.parametrize('rebase', ["CORRECTED","EXCEPTION"]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_ts_write_fails_datetime_exception(spark_tmp_path, ts_write_data_gen, spark_tmp_table_factory, rebase): ts_write, gen = ts_write_data_gen data_path = spark_tmp_path + '/PARQUET_DATA' @@ -471,7 +469,7 @@ def generate_map_with_empty_validity(spark, path): @pytest.mark.parametrize('data_gen', parquet_nested_datetime_gen, ids=idfn) @pytest.mark.parametrize('ts_write', parquet_ts_write_options) @pytest.mark.parametrize('ts_rebase_write', ['EXCEPTION']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_parquet_write_fails_legacy_datetime(spark_tmp_path, data_gen, ts_write, ts_rebase_write): data_path = spark_tmp_path + '/PARQUET_DATA' all_confs = {'spark.sql.parquet.outputTimestampType': ts_write, @@ -489,7 +487,7 @@ def writeParquetCatchException(spark, data_gen, data_path): @pytest.mark.parametrize('ts_write', parquet_ts_write_options) @pytest.mark.parametrize('ts_rebase_write', [('CORRECTED', 'LEGACY'), ('LEGACY', 'CORRECTED')]) @pytest.mark.parametrize('ts_rebase_read', [('CORRECTED', 'LEGACY'), ('LEGACY', 'CORRECTED')]) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_parquet_write_roundtrip_datetime_with_legacy_rebase(spark_tmp_path, data_gen, ts_write, ts_rebase_write, ts_rebase_read): data_path = spark_tmp_path + '/PARQUET_DATA' @@ -533,7 +531,7 @@ def test_it(spark): with_gpu_session(test_it, conf) @pytest.mark.parametrize('parquet_gens', parquet_write_gens_list, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_write_empty_parquet_round_trip(spark_tmp_path, parquet_gens): def create_empty_df(spark, path): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] @@ -773,7 +771,7 @@ def read_table(spark, path): # Test to avoid regression on a known bug in Spark. For details please visit https://github.com/NVIDIA/spark-rapids/issues/8693 @pytest.mark.parametrize('ts_rebase', ['LEGACY', 'CORRECTED']) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hive_timestamp_value(spark_tmp_table_factory, spark_tmp_path, ts_rebase): def func_test(create_table, read_table, data_path, conf): assert_gpu_and_cpu_writes_are_equal_collect(create_table, read_table, data_path, conf=conf) diff --git a/integration_tests/src/main/python/qa_nightly_select_test.py b/integration_tests/src/main/python/qa_nightly_select_test.py index d1a894e8e00..f783ca02518 100644 --- a/integration_tests/src/main/python/qa_nightly_select_test.py +++ b/integration_tests/src/main/python/qa_nightly_select_test.py @@ -24,8 +24,8 @@ from qa_nightly_sql import * import pytest from spark_session import with_cpu_session, is_jvm_charset_utf8 -from marks import approximate_float, ignore_order, incompat, qarun -from data_gen import copy_and_update +from marks import approximate_float, ignore_order, incompat, qarun, allow_non_gpu +from data_gen import * def num_stringDf(spark): print("### CREATE DATAFRAME 1 ####") @@ -159,7 +159,7 @@ def idfn(val): @incompat @qarun @pytest.mark.parametrize('sql_query_line', SELECT_SQL, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_select(sql_query_line, pytestconfig): sql_query = sql_query_line[0] if sql_query: @@ -172,7 +172,7 @@ def test_select(sql_query_line, pytestconfig): @incompat @qarun @pytest.mark.parametrize('sql_query_line', SELECT_NEEDS_SORT_SQL, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_needs_sort_select(sql_query_line, pytestconfig): sql_query = sql_query_line[0] if sql_query: diff --git a/integration_tests/src/main/python/repart_test.py b/integration_tests/src/main/python/repart_test.py index 28782e77ceb..54a540ce9a1 100644 --- a/integration_tests/src/main/python/repart_test.py +++ b/integration_tests/src/main/python/repart_test.py @@ -166,14 +166,14 @@ def test_union_by_name(data_gen): pytest.param([('array' + str(i), gen) for i, gen in enumerate(array_gens_sample + [ArrayGen(BinaryGen(max_length=5), max_length=5)])]), pytest.param([('map' + str(i), gen) for i, gen in enumerate(map_gens_sample)]), ], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_coalesce_types(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen).coalesce(2)) @pytest.mark.parametrize('num_parts', [1, 10, 100, 1000, 2000], ids=idfn) @pytest.mark.parametrize('length', [0, 2048, 4096], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_coalesce_df(num_parts, length): #This should change eventually to be more than just the basic gens gen_list = [('_c' + str(i), gen) for i, gen in enumerate(all_basic_gens + decimal_gens + [binary_gen])] @@ -189,7 +189,7 @@ def test_coalesce_df(num_parts, length): @pytest.mark.parametrize('num_parts', [1, 10, 2345], ids=idfn) @pytest.mark.parametrize('length', [0, 2048, 4096], ids=idfn) @ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test. -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_repartition_df(data_gen, num_parts, length): from pyspark.sql.functions import lit assert_gpu_and_cpu_are_equal_collect( @@ -206,7 +206,7 @@ def test_repartition_df(data_gen, num_parts, length): @pytest.mark.parametrize('num_parts', [1, 10, 2345], ids=idfn) @pytest.mark.parametrize('length', [0, 2048, 4096], ids=idfn) @ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test. -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_repartition_df_for_round_robin(data_gen, num_parts, length): from pyspark.sql.functions import lit assert_gpu_and_cpu_are_equal_collect( @@ -280,7 +280,7 @@ def test_hash_fallback(data_gen): ([('a', decimal_gen_64bit), ('b', decimal_gen_64bit), ('c', decimal_gen_64bit)], ['a', 'b', 'c']), ([('a', decimal_gen_128bit), ('b', decimal_gen_128bit), ('c', decimal_gen_128bit)], ['a', 'b', 'c']), ], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_hash_repartition_exact(gen, num_parts): data_gen = gen[0] part_on = gen[1] diff --git a/integration_tests/src/main/python/sort_test.py b/integration_tests/src/main/python/sort_test.py index 7a8d1c58612..3e447a5e772 100644 --- a/integration_tests/src/main/python/sort_test.py +++ b/integration_tests/src/main/python/sort_test.py @@ -194,13 +194,13 @@ def test_single_nested_sort_in_part(data_gen, order, stable_sort): boolean_gen, timestamp_gen, date_gen, string_gen, null_gen, StructGen([('child0', long_gen)]) ] + orderable_decimal_gens + single_level_array_gens @pytest.mark.parametrize('data_gen', orderable_gens_sort, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_multi_orderby(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).orderBy(f.col('a'), f.col('b').desc())) @pytest.mark.parametrize('data_gen', single_level_array_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_multi_orderby_on_array(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).orderBy(f.col('a'), f.col('b').desc())) @@ -208,7 +208,7 @@ def test_multi_orderby_on_array(data_gen): # SPARK CPU itself has issue with negative scale for take ordered and project orderable_gens_sort_without_neg_decimal = [n for n in orderable_gens_sort if not (isinstance(n, DecimalGen) and n.scale < 0)] @pytest.mark.parametrize('data_gen', orderable_gens_sort_without_neg_decimal + single_level_array_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_multi_orderby_with_limit(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).orderBy(f.col('a'), f.col('b').desc()).limit(100)) @@ -216,7 +216,7 @@ def test_multi_orderby_with_limit(data_gen): # We added in a partitioning optimization to take_ordered_and_project # This should trigger it. @pytest.mark.parametrize('data_gen', orderable_gens_sort_without_neg_decimal + single_level_array_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_multi_orderby_with_limit_single_part(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).coalesce(1).orderBy(f.col('a'), f.col('b').desc()).limit(100)) @@ -261,7 +261,7 @@ def test_single_orderby_with_skew(data_gen): # We are not trying all possibilities, just doing a few with numbers so the query works. @pytest.mark.parametrize('data_gen', [all_basic_struct_gen, StructGen([['child0', all_basic_struct_gen]])], ids=idfn) @pytest.mark.parametrize('stable_sort', ['STABLE', 'OUTOFCORE'], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_single_nested_orderby_with_skew(data_gen, stable_sort): sort_conf = {'spark.rapids.sql.stableSort.enabled': stable_sort == 'STABLE'} # When doing range partitioning the upstream data is sampled to try and get the bounds for cutoffs. @@ -305,7 +305,7 @@ def test_large_orderby(data_gen, stable_sort): simple_string_to_string_map_gen, ArrayGen(byte_gen, max_length=5)], ids=idfn) @pytest.mark.order(2) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_large_orderby_nested_ridealong(data_gen): # We use a UniqueLongGen to avoid duplicate keys that can cause ambiguity in the sort # results, especially on distributed clusters. @@ -326,7 +326,7 @@ def test_large_orderby_nested_ridealong(data_gen): ArrayGen(byte_gen, max_length=5), ArrayGen(decimal_gen_128bit, max_length=5)], ids=idfn) @pytest.mark.order(2) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_orderby_nested_ridealong_limit(data_gen): # We use a UniqueLongGen to avoid duplicate keys that can cause ambiguity in the sort # results, especially on distributed clusters. diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index 81ce8d74948..681f7d3d6f1 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -523,7 +523,7 @@ def test_rows_based_running_window_unpartitioned(b_gen, batch_size): @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # Testing multiple batch sizes. @pytest.mark.parametrize('a_gen', integral_gens + [string_gen, date_gen, timestamp_gen], ids=meta_idfn('data:')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_running_window_without_partitions_runs_batched(a_gen, batch_size): """ This tests the running window optimization as applied to RANGE-based window specifications, @@ -647,7 +647,7 @@ def test_running_window_float_sum_without_partitions_runs_batched(batch_size): @pytest.mark.parametrize('data_gen', all_basic_gens + [decimal_gen_32bit, orderable_decimal_gen_128bit], ids=meta_idfn('data:')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_window_running_rank_no_part(data_gen): # Keep the batch size small. We have tested these with operators with exact inputs already, this is mostly # testing the fixup operation. @@ -675,7 +675,7 @@ def test_window_running_rank_no_part(data_gen): # but small batch sizes can make sort very slow, so do the final order by locally @ignore_order(local=True) @pytest.mark.parametrize('data_gen', all_basic_gens + [decimal_gen_32bit], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_window_running_rank(data_gen): # Keep the batch size small. We have tested these with operators with exact inputs already, this is mostly # testing the fixup operation. @@ -703,7 +703,7 @@ def test_window_running_rank(data_gen): @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches @pytest.mark.parametrize('b_gen, c_gen', [(long_gen, x) for x in running_part_and_order_gens] + [(x, long_gen) for x in all_basic_gens + [decimal_gen_32bit]], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_rows_based_running_window_partitioned(b_gen, c_gen, batch_size): conf = {'spark.rapids.sql.batchSizeBytes': batch_size, 'spark.rapids.sql.variableFloatAgg.enabled': True, @@ -743,7 +743,7 @@ def test_rows_based_running_window_partitioned(b_gen, c_gen, batch_size): @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # Test different batch sizes. @pytest.mark.parametrize('part_gen', [int_gen, long_gen], ids=idfn) # Partitioning is not really the focus of the test. @pytest.mark.parametrize('order_gen', [x for x in all_basic_gens_no_null if x not in boolean_gens] + [decimal_gen_32bit], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_range_running_window_runs_batched(part_gen, order_gen, batch_size): """ This tests the running window optimization as applied to RANGE-based window specifications, @@ -887,7 +887,7 @@ def window(oby_column): @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches @pytest.mark.parametrize('c_gen', lead_lag_data_gens, ids=idfn) @pytest.mark.parametrize('a_b_gen', part_and_order_gens, ids=meta_idfn('partAndOrderBy:')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_multi_types_window_aggs_for_rows_lead_lag(a_b_gen, c_gen, batch_size): conf = {'spark.rapids.sql.batchSizeBytes': batch_size} data_gen = [ @@ -945,7 +945,7 @@ def do_it(spark): @approximate_float @pytest.mark.parametrize('struct_gen', lead_lag_struct_with_arrays_gen, ids=idfn) @pytest.mark.parametrize('a_b_gen', part_and_order_gens, ids=meta_idfn('partAndOrderBy:')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_lead_lag_for_structs_with_arrays(a_b_gen, struct_gen): data_gen = [ ('a', RepeatSeqGen(a_b_gen, length=20)), @@ -979,7 +979,7 @@ def do_it(spark): @pytest.mark.parametrize('c_gen', [UniqueLongGen()], ids=meta_idfn('orderBy:')) @pytest.mark.parametrize('b_gen', [long_gen], ids=meta_idfn('orderBy:')) @pytest.mark.parametrize('a_gen', [long_gen], ids=meta_idfn('partBy:')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_window_aggs_for_rows_lead_lag_on_arrays(a_gen, b_gen, c_gen, d_gen): data_gen = [ ('a', RepeatSeqGen(a_gen, length=20)), @@ -1009,7 +1009,7 @@ def test_window_aggs_for_rows_lead_lag_on_arrays(a_gen, b_gen, c_gen, d_gen): @approximate_float @pytest.mark.parametrize('c_gen', [string_gen], ids=idfn) @pytest.mark.parametrize('a_b_gen', part_and_order_gens, ids=meta_idfn('partAndOrderBy:')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_multi_types_window_aggs_for_rows(a_b_gen, c_gen): data_gen = [ ('a', RepeatSeqGen(a_b_gen, length=20)), @@ -1061,13 +1061,12 @@ def do_it(spark): assert_gpu_and_cpu_are_equal_collect(do_it, conf = {'spark.rapids.sql.batchSizeBytes': '100'}) @pytest.mark.skipif(is_before_spark_320(), reason="Only in Spark 3.2.0 is IGNORE NULLS supported for lead and lag by Spark") -@allow_non_gpu('WindowExec', 'Alias', 'WindowExpression', 'Lead', 'Literal', 'WindowSpecDefinition', 'SpecifiedWindowFrame') +@allow_non_gpu('WindowExec', 'Alias', 'WindowExpression', 'Lead', 'Literal', 'WindowSpecDefinition', 'SpecifiedWindowFrame', *non_utc_allow) @ignore_order(local=True) @pytest.mark.parametrize('d_gen', all_basic_gens, ids=meta_idfn('agg:')) @pytest.mark.parametrize('c_gen', [UniqueLongGen()], ids=meta_idfn('orderBy:')) @pytest.mark.parametrize('b_gen', [long_gen], ids=meta_idfn('orderBy:')) @pytest.mark.parametrize('a_gen', [long_gen], ids=meta_idfn('partBy:')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_window_aggs_lead_ignore_nulls_fallback(a_gen, b_gen, c_gen, d_gen): data_gen = [ ('a', RepeatSeqGen(a_gen, length=20)), @@ -1086,13 +1085,12 @@ def test_window_aggs_lead_ignore_nulls_fallback(a_gen, b_gen, c_gen, d_gen): ''') @pytest.mark.skipif(is_before_spark_320(), reason="Only in Spark 3.2.0 is IGNORE NULLS supported for lead and lag by Spark") -@allow_non_gpu('WindowExec', 'Alias', 'WindowExpression', 'Lag', 'Literal', 'WindowSpecDefinition', 'SpecifiedWindowFrame') +@allow_non_gpu('WindowExec', 'Alias', 'WindowExpression', 'Lag', 'Literal', 'WindowSpecDefinition', 'SpecifiedWindowFrame', *non_utc_allow) @ignore_order(local=True) @pytest.mark.parametrize('d_gen', all_basic_gens, ids=meta_idfn('agg:')) @pytest.mark.parametrize('c_gen', [UniqueLongGen()], ids=meta_idfn('orderBy:')) @pytest.mark.parametrize('b_gen', [long_gen], ids=meta_idfn('orderBy:')) @pytest.mark.parametrize('a_gen', [long_gen], ids=meta_idfn('partBy:')) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') def test_window_aggs_lag_ignore_nulls_fallback(a_gen, b_gen, c_gen, d_gen): data_gen = [ ('a', RepeatSeqGen(a_gen, length=20)), @@ -1117,7 +1115,7 @@ def test_window_aggs_lag_ignore_nulls_fallback(a_gen, b_gen, c_gen, d_gen): @pytest.mark.parametrize('data_gen', [_grpkey_longs_with_timestamps, pytest.param(_grpkey_longs_with_nullable_timestamps)], ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_window_aggs_for_ranges_timestamps(data_gen): assert_gpu_and_cpu_are_equal_sql( lambda spark: gen_df(spark, data_gen, length=2048), @@ -1265,7 +1263,7 @@ def test_window_aggregations_for_big_decimal_ranges(data_gen): # SortExec does not support array type, so sort the result locally. @ignore_order(local=True) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_window_aggs_for_rows_collect_list(): assert_gpu_and_cpu_are_equal_sql( lambda spark : gen_df(spark, _gen_data_for_collect_list), @@ -1312,7 +1310,7 @@ def test_window_aggs_for_rows_collect_list(): @ignore_order(local=True) # This test is more directed at Databricks and their running window optimization instead of ours # this is why we do not validate that we inserted in a GpuRunningWindowExec, yet. -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_running_window_function_exec_for_all_aggs(): assert_gpu_and_cpu_are_equal_sql( lambda spark : gen_df(spark, _gen_data_for_collect_list), @@ -1421,7 +1419,7 @@ def do_it(spark): # SortExec does not support array type, so sort the result locally. @ignore_order(local=True) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_window_aggs_for_rows_collect_set(): assert_gpu_and_cpu_are_equal_sql( lambda spark: gen_df(spark, _gen_data_for_collect_set), @@ -1482,8 +1480,7 @@ def test_window_aggs_for_rows_collect_set(): # See https://github.com/NVIDIA/spark-rapids/issues/3715 # and https://github.com/rapidsai/cudf/issues/11222 @ignore_order(local=True) -@allow_non_gpu("ProjectExec", "SortArray") -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu("ProjectExec", "SortArray", *non_utc_allow) def test_window_aggs_for_rows_collect_set_nested_array(): conf = copy_and_update(_float_conf, { "spark.rapids.sql.castFloatToString.enabled": "true", @@ -1596,7 +1593,7 @@ def do_it(spark): # but small batch sizes can make sort very slow, so do the final order by locally @ignore_order(local=True) @pytest.mark.parametrize('ride_along', all_basic_gens + decimal_gens + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_window_ride_along(ride_along): assert_gpu_and_cpu_are_equal_sql( lambda spark : gen_df(spark, [('a', UniqueLongGen()), ('b', ride_along)]), @@ -1672,7 +1669,7 @@ def test_unbounded_to_unbounded_window(): 'last(a) IGNORE NULLS OVER (PARTITION BY b ORDER BY c) ' @pytest.mark.parametrize('data_gen', all_basic_gens_no_null + decimal_gens + _nested_gens, ids=idfn) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_window_first_last_nth(data_gen): assert_gpu_and_cpu_are_equal_sql( # Coalesce is to make sure that first and last, which are non-deterministic become deterministic @@ -1693,7 +1690,7 @@ def test_window_first_last_nth_ignore_nulls(data_gen): @ignore_order(local=True) -@pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@allow_non_gpu(*non_utc_allow) def test_to_date_with_window_functions(): """ This test ensures that date expressions participating alongside window aggregations From 568f8e2daf32c2ef822f6f40710cbe6a6f37f757 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 09:06:37 +0800 Subject: [PATCH 22/46] Fix CSV scan --- .../src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala index 97138843065..ab9dc6748ff 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala @@ -185,6 +185,10 @@ object GpuCSVScan { if (types.contains(TimestampType)) { GpuTextBasedDateUtils.tagCudfFormat(meta, GpuCsvUtils.timestampFormatInRead(parsedOptions), parseString = true) + + if (!TimeZoneDB.isUTCTimezone(parsedOptions.zoneId)) { + meta.willNotWorkOnGpu(s"Not supported timezone type ${parsedOptions.zoneId}.") + } } // TODO parsedOptions.emptyValueInRead From a743a7a6c226518b94769f2d499854aa09db603a Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Tue, 28 Nov 2023 21:31:38 +0800 Subject: [PATCH 23/46] Fix --- integration_tests/src/main/python/data_gen.py | 2 +- integration_tests/src/main/python/parquet_testing_test.py | 2 +- integration_tests/src/main/python/qa_nightly_select_test.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index ca2619ca0e3..da81309c69e 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -1175,4 +1175,4 @@ def get_25_partitions_df(spark): # allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' -non_utc_allow=['ProjectExec', 'FilterExec', 'FileSourceScanExec', 'BatchScanExec', 'CollectLimitExec', 'DeserializeToObjectExec', 'DataWritingCommandExec', 'WriteFilesExec', 'ShuffleExchangeExec', 'ObjectHashAggregateExec'] if is_not_utc() else [] \ No newline at end of file +non_utc_allow=['ProjectExec', 'FilterExec', 'FileSourceScanExec', 'BatchScanExec', 'CollectLimitExec', 'DeserializeToObjectExec', 'DataWritingCommandExec', 'WriteFilesExec', 'ShuffleExchangeExec'] if is_not_utc() else [] \ No newline at end of file diff --git a/integration_tests/src/main/python/parquet_testing_test.py b/integration_tests/src/main/python/parquet_testing_test.py index abf06d11e5f..6c3ab0c14a2 100644 --- a/integration_tests/src/main/python/parquet_testing_test.py +++ b/integration_tests/src/main/python/parquet_testing_test.py @@ -17,7 +17,7 @@ from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error from conftest import get_std_input_path, is_parquet_testing_tests_forced, is_precommit_run, is_not_utc -from data_gen import * +from data_gen import copy_and_update, non_utc_allow from marks import allow_non_gpu from pathlib import Path import pytest diff --git a/integration_tests/src/main/python/qa_nightly_select_test.py b/integration_tests/src/main/python/qa_nightly_select_test.py index f783ca02518..786b71face5 100644 --- a/integration_tests/src/main/python/qa_nightly_select_test.py +++ b/integration_tests/src/main/python/qa_nightly_select_test.py @@ -25,7 +25,7 @@ import pytest from spark_session import with_cpu_session, is_jvm_charset_utf8 from marks import approximate_float, ignore_order, incompat, qarun, allow_non_gpu -from data_gen import * +from data_gen import copy_and_update, non_utc_allow def num_stringDf(spark): print("### CREATE DATAFRAME 1 ####") From 49424969ee8bdf5e5c637b65c2ab7e33171ffdcc Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 10:45:11 +0800 Subject: [PATCH 24/46] Fix explain on CPU --- integration_tests/src/main/python/explain_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/explain_test.py b/integration_tests/src/main/python/explain_test.py index d182c093850..84dbd4c5473 100644 --- a/integration_tests/src/main/python/explain_test.py +++ b/integration_tests/src/main/python/explain_test.py @@ -50,7 +50,7 @@ def do_join_explain(spark): with_cpu_session(do_join_explain) -@allow_non_gpu(*non_utc_allow) +@pytest.mark.skipif(is_not_utc(), reason='Cast is not supported with timezone setting. https://github.com/NVIDIA/spark-rapids/issues/9653') def test_explain_set_config(): conf = {'spark.rapids.sql.hasExtendedYearValues': 'false', 'spark.rapids.sql.castStringToTimestamp.enabled': 'true'} From 83171ae1ad61f8069ea1a0f4b33e73e908381b8f Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 10:57:39 +0800 Subject: [PATCH 25/46] Fix json --- integration_tests/src/main/python/json_test.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 2b4161b3dec..f2916966267 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -356,6 +356,9 @@ def test_json_read_valid_dates(std_input_path, filename, schema, read_func, ansi else: assert_gpu_and_cpu_are_equal_collect(f, conf=updated_conf) +# allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' +not_utc_json_scan_allow=['FileSourceScanExec'] if is_not_utc() else [] + @approximate_float @pytest.mark.parametrize('filename', [ 'dates_invalid.json', @@ -365,8 +368,8 @@ def test_json_read_valid_dates(std_input_path, filename, schema, read_func, ansi @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) @pytest.mark.parametrize('time_parser_policy', [ pytest.param('LEGACY', marks=pytest.mark.allow_non_gpu('FileSourceScanExec')), - 'CORRECTED', - 'EXCEPTION' + pytest.param('CORRECTED', marks=pytest.mark.allow_non_gpu(*not_utc_json_scan_allow)), + pytest.param('EXCEPTION', marks=pytest.mark.allow_non_gpu(*not_utc_json_scan_allow)) ]) def test_json_read_invalid_dates(std_input_path, filename, schema, read_func, ansi_enabled, time_parser_policy, spark_tmp_table_factory): updated_conf = copy_and_update(_enable_all_types_conf, From e9b3b10fe454f69b8dc5fde41953718cf1acf4ef Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 11:01:19 +0800 Subject: [PATCH 26/46] Fix json --- integration_tests/src/main/python/json_test.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index f2916966267..116b9d6b3e1 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -331,6 +331,10 @@ def test_read_valid_json(spark_tmp_table_factory, std_input_path, read_func, fil {}), conf=conf) + +# allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' +not_utc_json_scan_allow=['FileSourceScanExec'] if is_not_utc() else [] + @approximate_float @pytest.mark.parametrize('filename', [ 'dates.json', @@ -340,8 +344,8 @@ def test_read_valid_json(spark_tmp_table_factory, std_input_path, read_func, fil @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) @pytest.mark.parametrize('time_parser_policy', [ pytest.param('LEGACY', marks=pytest.mark.allow_non_gpu('FileSourceScanExec')), - 'CORRECTED', - 'EXCEPTION' + pytest.param('CORRECTED', marks=pytest.mark.allow_non_gpu(*not_utc_json_scan_allow)), + pytest.param('EXCEPTION', marks=pytest.mark.allow_non_gpu(*not_utc_json_scan_allow)) ]) def test_json_read_valid_dates(std_input_path, filename, schema, read_func, ansi_enabled, time_parser_policy, spark_tmp_table_factory): updated_conf = copy_and_update(_enable_all_types_conf, @@ -356,8 +360,6 @@ def test_json_read_valid_dates(std_input_path, filename, schema, read_func, ansi else: assert_gpu_and_cpu_are_equal_collect(f, conf=updated_conf) -# allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' -not_utc_json_scan_allow=['FileSourceScanExec'] if is_not_utc() else [] @approximate_float @pytest.mark.parametrize('filename', [ From a5648917243bf0c30c1aceffe833243696d70d33 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 11:57:12 +0800 Subject: [PATCH 27/46] Fix ORC scan --- integration_tests/src/main/python/orc_cast_test.py | 3 ++- .../scala/com/nvidia/spark/rapids/GpuOrcScan.scala | 14 ++++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/orc_cast_test.py b/integration_tests/src/main/python/orc_cast_test.py index 5f6838df688..4367357ba82 100644 --- a/integration_tests/src/main/python/orc_cast_test.py +++ b/integration_tests/src/main/python/orc_cast_test.py @@ -49,9 +49,10 @@ def test_casting_among_integer_types(spark_tmp_path, reader_confs, v1_enabled_li lambda spark: spark.read.schema(schema_str).orc(data_path), conf=all_confs) +non_utc_allow_orc_scan=['ColumnarToRowExec', 'FileScan'] if is_not_utc() else [] @pytest.mark.parametrize('to_type', ['float', 'double', 'string', 'timestamp']) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_orc_scan) def test_casting_from_integer(spark_tmp_path, to_type): orc_path = spark_tmp_path + '/orc_cast_integer' # The Python 'datetime' module only supports a max-year of 10000, so we set the Long type max diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index abbda8e303b..d3e82a0637b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -21,6 +21,7 @@ import java.net.URI import java.nio.ByteBuffer import java.nio.channels.Channels import java.nio.charset.StandardCharsets +import java.time.ZoneId import java.util import java.util.concurrent.{Callable, TimeUnit} import java.util.regex.Pattern @@ -154,6 +155,19 @@ object GpuOrcScan { meta.willNotWorkOnGpu("GpuOrcScan does not support default values in schema") } + // For date type, timezone needs to be checked also. This is because JVM timezone and UTC + // timezone offset is considered when getting [[java.sql.date]] from + // [[org.apache.spark.sql.execution.datasources.DaysWritable]] object + // which is a subclass of [[org.apache.hadoop.hive.serde2.io.DateWritable]]. + val types = schema.map(_.dataType).toSet + if (types.exists(GpuOverrides.isOrContainsDateOrTimestamp(_))) { + if (!GpuOverrides.isUTCTimezone()) { + meta.willNotWorkOnGpu("Only UTC timezone is supported for ORC. " + + s"Current timezone settings: (JVM : ${ZoneId.systemDefault()}, " + + s"session: ${SQLConf.get.sessionLocalTimeZone}). ") + } + } + FileFormatChecks.tag(meta, schema, OrcFormatType, ReadFileOp) } From f497490bdad1d1ad71215756ee22e9c994676aac Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 12:02:20 +0800 Subject: [PATCH 28/46] Fix ORC test --- .../src/main/python/orc_cast_test.py | 2 +- integration_tests/src/main/python/orc_test.py | 21 +++++++++++-------- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/integration_tests/src/main/python/orc_cast_test.py b/integration_tests/src/main/python/orc_cast_test.py index 4367357ba82..f358c2f6207 100644 --- a/integration_tests/src/main/python/orc_cast_test.py +++ b/integration_tests/src/main/python/orc_cast_test.py @@ -105,7 +105,7 @@ def test_casting_from_float_and_double(spark_tmp_path, to_type): @pytest.mark.parametrize('data_gen', [DoubleGen(max_exp=32, special_cases=None), DoubleGen(max_exp=32, special_cases=[8.88e9, 9.99e10, 1.314e11])]) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_orc_scan) def test_casting_from_double_to_timestamp(spark_tmp_path, data_gen): # ORC will assume the original double value in seconds, we need to convert them to # timestamp(INT64 in micro-seconds). diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index f539e5f6c74..c3bbd917b2e 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -62,13 +62,14 @@ def get_orc_timestamp_gen(nullable=True): # The Count result can not be sorted, so local sort can not be used. reader_opt_confs_for_count = reader_opt_confs_common + [multithreaded_orc_file_reader_combine_unordered_conf] +non_utc_allow_orc_file_source_scan=['ColumnarToRowExec', 'FileSourceScanExec'] if is_not_utc() else [] @pytest.mark.parametrize('name', ['timestamp-date-test.orc']) @pytest.mark.parametrize('read_func', [read_orc_df, read_orc_sql]) @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('orc_impl', ["native", "hive"]) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_orc_file_source_scan) def test_basic_read(std_input_path, name, read_func, v1_enabled_list, orc_impl, reader_confs): all_confs = copy_and_update(reader_confs, { 'spark.sql.sources.useV1SourceList': v1_enabled_list, @@ -127,6 +128,8 @@ def test_basic_read(std_input_path, name, read_func, v1_enabled_list, orc_impl, MapGen(StructGen([['child0', byte_gen], ['child1', long_gen]], nullable=False), StructGen([['child0', byte_gen], ['child1', long_gen]]))] +non_utc_allow_orc_scan=['ColumnarToRowExec', 'FileSourceScanExec', 'BatchScanExec'] if is_not_utc() else [] + orc_gens_list = [orc_basic_gens, orc_array_gens_sample, orc_struct_gens_sample, @@ -159,8 +162,8 @@ def test_orc_fallback(spark_tmp_path, read_func, disable_conf): @pytest.mark.parametrize('orc_gens', orc_gens_list, ids=idfn) @pytest.mark.parametrize('read_func', [read_orc_df, read_orc_sql]) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) -@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) -@allow_non_gpu(*non_utc_allow) +@pytest.mark.parametrize('v1_enabled_list', ['', 'orc']) +@allow_non_gpu(*non_utc_allow_orc_scan) def test_read_round_trip(spark_tmp_path, orc_gens, read_func, reader_confs, v1_enabled_list): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)] data_path = spark_tmp_path + '/ORC_DATA' @@ -186,7 +189,7 @@ def test_read_round_trip(spark_tmp_path, orc_gens, read_func, reader_confs, v1_e @pytest.mark.parametrize('read_func', [read_orc_df, read_orc_sql]) @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_orc_scan) def test_pred_push_round_trip(spark_tmp_path, orc_gen, read_func, v1_enabled_list, reader_confs): data_path = spark_tmp_path + '/ORC_DATA' # Append two struct columns to verify nested predicate pushdown. @@ -243,7 +246,7 @@ def test_compress_read_round_trip(spark_tmp_path, compress, v1_enabled_list, rea @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_orc_scan) def test_simple_partitioned_read(spark_tmp_path, v1_enabled_list, reader_confs): # Once https://github.com/NVIDIA/spark-rapids/issues/131 is fixed # we should go with a more standard set of generators @@ -310,7 +313,7 @@ def test_partitioned_read_just_partitions(spark_tmp_path, v1_enabled_list, reade @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_orc_scan) def test_merge_schema_read(spark_tmp_path, v1_enabled_list, reader_confs): # Once https://github.com/NVIDIA/spark-rapids/issues/131 is fixed # we should go with a more standard set of generators @@ -589,7 +592,7 @@ def test_read_struct_without_stream(spark_tmp_path): @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('case_sensitive', ["false", "true"]) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_orc_scan) def test_read_with_more_columns(spark_tmp_path, orc_gen, reader_confs, v1_enabled_list, case_sensitive): struct_gen = StructGen([('nested_col', orc_gen)]) # Map is not supported yet. @@ -777,7 +780,7 @@ def test_orc_read_varchar_as_string(std_input_path): @pytest.mark.parametrize('gens', orc_gens_list, ids=idfn) @pytest.mark.parametrize('keep_order', [True, pytest.param(False, marks=pytest.mark.ignore_order(local=True))]) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_orc_scan) def test_read_round_trip_for_multithreaded_combining(spark_tmp_path, gens, keep_order): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(gens)] data_path = spark_tmp_path + '/ORC_DATA' @@ -792,7 +795,7 @@ def test_read_round_trip_for_multithreaded_combining(spark_tmp_path, gens, keep_ @pytest.mark.parametrize('keep_order', [True, pytest.param(False, marks=pytest.mark.ignore_order(local=True))]) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_orc_scan) def test_simple_partitioned_read_for_multithreaded_combining(spark_tmp_path, keep_order): orc_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), From 63d9e261d0edccd2824967516a87476cd019b32b Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 14:21:41 +0800 Subject: [PATCH 29/46] skip legacy mode rebase --- integration_tests/src/main/python/parquet_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py index 5d490f71029..ae3aeab7410 100644 --- a/integration_tests/src/main/python/parquet_test.py +++ b/integration_tests/src/main/python/parquet_test.py @@ -559,7 +559,7 @@ def test_parquet_read_merge_schema(spark_tmp_path, v1_enabled_list, reader_confs @pytest.mark.parametrize('reader_confs', reader_opt_confs) @pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) -@allow_non_gpu(*non_utc_allow) +@pytest.mark.skipif(is_not_utc(), reason="LEGACY datetime rebase mode is only supported for UTC timezone") def test_parquet_read_merge_schema_from_conf(spark_tmp_path, v1_enabled_list, reader_confs): # Once https://github.com/NVIDIA/spark-rapids/issues/133 and https://github.com/NVIDIA/spark-rapids/issues/132 are fixed # we should go with a more standard set of generators From 1cbb694e9c33b1a068df3419933b2f61650f0d79 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 16:39:23 +0800 Subject: [PATCH 30/46] Support check for AnsiCast --- .../src/main/python/parquet_write_test.py | 5 +++-- .../scala/com/nvidia/spark/rapids/PlanShims.scala | 13 +++++++++++++ .../scala/com/nvidia/spark/rapids/RapidsMeta.scala | 9 ++++++--- .../nvidia/spark/rapids/shims/AnsiCastShim.scala | 6 ++++++ .../nvidia/spark/rapids/shims/PlanShimsImpl.scala | 5 +++++ .../nvidia/spark/rapids/shims/PlanShimsImpl.scala | 5 +++++ .../nvidia/spark/rapids/shims/AnsiCastShim.scala | 6 ++++++ .../nvidia/spark/rapids/shims/PlanShimsImpl.scala | 5 +++++ 8 files changed, 49 insertions(+), 5 deletions(-) diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index 00079fffe0d..e25546fcd00 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -770,8 +770,9 @@ def read_table(spark, path): func(create_table, read_table, data_path, conf) # Test to avoid regression on a known bug in Spark. For details please visit https://github.com/NVIDIA/spark-rapids/issues/8693 -@pytest.mark.parametrize('ts_rebase', ['LEGACY', 'CORRECTED']) -@allow_non_gpu(*non_utc_allow) +@pytest.mark.parametrize('ts_rebase', [ + pytest.param('LEGACY', marks=pytest.mark.skipif(is_not_utc(), reason="LEGACY datetime rebase mode is only supported for UTC timezone")), + 'CORRECTED']) def test_hive_timestamp_value(spark_tmp_table_factory, spark_tmp_path, ts_rebase): def func_test(create_table, read_table, data_path, conf): assert_gpu_and_cpu_writes_are_equal_collect(create_table, read_table, data_path, conf=conf) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PlanShims.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PlanShims.scala index cfb1586d6e3..dcab3e53f20 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PlanShims.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PlanShims.scala @@ -18,11 +18,19 @@ package com.nvidia.spark.rapids import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types.DataType trait PlanShims { def extractExecutedPlan(plan: SparkPlan): SparkPlan def isAnsiCast(e: Expression): Boolean def isAnsiCastOptionallyAliased(e: Expression): Boolean + + /** + * Extra Ansi Cast's source's data type and target's data type + * @param e should be AnsiCast type or Cast under Ansi mode + * @return (source data type, target data type) + */ + def extractAnsiCastTypes(e: Expression): (DataType, DataType) } object PlanShims { @@ -35,7 +43,12 @@ object PlanShims { def isAnsiCast(e: Expression): Boolean = { shims.isAnsiCast(e) } + def isAnsiCastOptionallyAliased(e: Expression): Boolean = { shims.isAnsiCastOptionallyAliased(e) } + + def extractAnsiCastTypes(e: Expression): (DataType, DataType) = { + shims.extractAnsiCastTypes(e) + } } \ No newline at end of file diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index fd0075b3869..65544b1a4e4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -1135,13 +1135,16 @@ abstract class BaseExprMeta[INPUT <: Expression]( //+------------------------+-------------------+-----------------------------------------+ lazy val needTimeZoneCheck: Boolean = { wrapped match { + // CurrentDate expression will not go through this even it's a `TimeZoneAwareExpression`. + // It will be treated as literal in Rapids. case _: TimeZoneAwareExpression => - // CurrentDate expression will not go through this even it's a `TimeZoneAwareExpression`. - // It will be treated as literal in Rapids. if (wrapped.isInstanceOf[Cast]) { val cast = wrapped.asInstanceOf[Cast] needsTimeZone(cast.child.dataType, cast.dataType) - } else { + } else if(PlanShims.isAnsiCast(wrapped)) { + val (from, to) = PlanShims.extractAnsiCastTypes(wrapped) + needsTimeZone(from, to) + } else{ true } case _: UTCTimestamp => true diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala index d879b6da1e9..e67425d2591 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala @@ -37,6 +37,7 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids.{GpuCast, GpuEvalMode} import org.apache.spark.sql.catalyst.expressions.{AnsiCast, Cast, Expression} +import org.apache.spark.sql.types.DataType object AnsiCastShim { def isAnsiCast(e: Expression): Boolean = e match { @@ -59,4 +60,9 @@ object AnsiCastShim { m.setAccessible(true) m.getBoolean(e) } + + def extractAnsiCastTypes(e: Expression): (DataType, DataType) = e match { + case c: AnsiCast => (c.child.dataType, c.dataType) + case _ => throw new UnsupportedOperationException(s"${e.getClass} is not AnsiCast type") + } } \ No newline at end of file diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala index 5be71b13a51..438038fb749 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala @@ -25,6 +25,7 @@ import com.nvidia.spark.rapids.{GpuAlias, PlanShims} import org.apache.spark.sql.catalyst.expressions.{Alias, Expression} import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types.DataType class PlanShimsImpl extends PlanShims { def extractExecutedPlan(plan: SparkPlan): SparkPlan = plan @@ -35,4 +36,8 @@ class PlanShimsImpl extends PlanShims { case GpuAlias(e, _) => isAnsiCast(e) case e => isAnsiCast(e) } + + def extractAnsiCastTypes(e: Expression): (DataType, DataType) = { + AnsiCastShim.extractAnsiCastTypes(e) + } } diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala index 1ac7eeddf3b..7af5e029afb 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala @@ -40,6 +40,7 @@ import com.nvidia.spark.rapids.{GpuAlias, PlanShims} import org.apache.spark.sql.catalyst.expressions.{Alias, Expression} import org.apache.spark.sql.execution.{CommandResultExec, SparkPlan} +import org.apache.spark.sql.types.DataType class PlanShimsImpl extends PlanShims { def extractExecutedPlan(plan: SparkPlan): SparkPlan = plan match { @@ -54,4 +55,8 @@ class PlanShimsImpl extends PlanShims { case GpuAlias(e, _) => isAnsiCast(e) case e => isAnsiCast(e) } + + def extractAnsiCastTypes(e: Expression): (DataType, DataType) = { + AnsiCastShim.extractAnsiCastTypes(e) + } } diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala index a5b0c79766e..e82c1ae9aed 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala @@ -26,6 +26,7 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids.{GpuCast, GpuEvalMode} import org.apache.spark.sql.catalyst.expressions.{Cast, EvalMode, Expression} +import org.apache.spark.sql.types.DataType object AnsiCastShim { def isAnsiCast(e: Expression): Boolean = e match { @@ -41,4 +42,9 @@ object AnsiCastShim { case EvalMode.TRY => GpuEvalMode.TRY } } + + def extractAnsiCastTypes(e: Expression): (DataType, DataType) = e match { + case c: Cast => (c.child.dataType, c.dataType) + case _ => throw new UnsupportedOperationException(s"${e.getClass} is Cast type") + } } diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala index 2968081dff6..bf5567298bf 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala @@ -24,6 +24,7 @@ import com.nvidia.spark.rapids.{GpuAlias, PlanShims} import org.apache.spark.sql.catalyst.expressions.{Alias, Expression} import org.apache.spark.sql.execution.{CommandResultExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.ResultQueryStageExec +import org.apache.spark.sql.types.DataType class PlanShimsImpl extends PlanShims { def extractExecutedPlan(plan: SparkPlan): SparkPlan = plan match { @@ -39,4 +40,8 @@ class PlanShimsImpl extends PlanShims { case GpuAlias(e, _) => isAnsiCast(e) case e => isAnsiCast(e) } + + def extractAnsiCastTypes(e: Expression): (DataType, DataType) = { + AnsiCastShim.extractAnsiCastTypes(e) + } } From 07b6863a88629b6ab764f57f7f26ee07067a1f15 Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Wed, 29 Nov 2023 12:28:58 +0330 Subject: [PATCH 31/46] Fix cases --- integration_tests/src/main/python/aqe_test.py | 3 +++ integration_tests/src/main/python/csv_test.py | 7 +++++-- integration_tests/src/main/python/hash_aggregate_test.py | 6 ++---- .../src/main/python/hive_delimited_text_test.py | 7 ++++--- integration_tests/src/main/python/json_test.py | 2 ++ 5 files changed, 16 insertions(+), 9 deletions(-) diff --git a/integration_tests/src/main/python/aqe_test.py b/integration_tests/src/main/python/aqe_test.py index 7f2f68c5880..1ad8f4f5cb0 100755 --- a/integration_tests/src/main/python/aqe_test.py +++ b/integration_tests/src/main/python/aqe_test.py @@ -21,6 +21,9 @@ from marks import ignore_order, allow_non_gpu from spark_session import with_cpu_session, is_databricks113_or_later +# allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' +non_utc_allow=['HashAggregateExec', 'ProjectExec', 'FilterExec', 'FileSourceScanExec', 'BatchScanExec', 'CollectLimitExec', 'DeserializeToObjectExec', 'DataWritingCommandExec', 'WriteFilesExec', 'ShuffleExchangeExec'] if is_not_utc() else [] + _adaptive_conf = { "spark.sql.adaptive.enabled": "true" } def create_skew_df(spark, length): diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index c7330a06c33..d7836d77c5b 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -334,6 +334,7 @@ def test_csv_fallback(spark_tmp_path, read_func, disable_conf, spark_tmp_table_f 'CORRECTED', 'EXCEPTION' ]) +@allow_non_gpu(*non_utc_allow) def test_date_formats_round_trip(spark_tmp_path, date_format, v1_enabled_list, ansi_enabled, time_parser_policy): gen = StructGen([('a', DateGen())], nullable=False) data_path = spark_tmp_path + '/CSV_DATA' @@ -365,13 +366,15 @@ def test_date_formats_round_trip(spark_tmp_path, date_format, v1_enabled_list, a .csv(data_path), conf=updated_conf) + +non_utc_allow_for_test_read_valid_and_invalid_dates=['FileSourceScanExec', 'BatchScanExec'] if is_not_utc() else [] @pytest.mark.parametrize('filename', ["date.csv"]) @pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) @pytest.mark.parametrize('time_parser_policy', [ pytest.param('LEGACY', marks=pytest.mark.allow_non_gpu('BatchScanExec,FileSourceScanExec')), - 'CORRECTED', - 'EXCEPTION' + pytest.param('CORRECTED', marks=pytest.mark.allow_non_gpu(*non_utc_allow_for_test_read_valid_and_invalid_dates)), + pytest.param('EXCEPTION', marks=pytest.mark.allow_non_gpu(*non_utc_allow_for_test_read_valid_and_invalid_dates)) ]) def test_read_valid_and_invalid_dates(std_input_path, filename, v1_enabled_list, ansi_enabled, time_parser_policy): data_path = std_input_path + '/' + filename diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index a6f1bb017fe..c42b5dffaf7 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -810,12 +810,11 @@ def spark_fn(spark_session): @allow_non_gpu('ObjectHashAggregateExec', 'SortAggregateExec', 'ShuffleExchangeExec', 'HashPartitioning', 'SortExec', 'SortArray', 'Alias', 'Literal', 'Count', 'CollectList', 'CollectSet', - 'AggregateExpression', 'ProjectExec') + 'AggregateExpression', 'ProjectExec', *non_utc_allow) @pytest.mark.parametrize('data_gen', _full_gen_data_for_collect_op, ids=idfn) @pytest.mark.parametrize('replace_mode', _replace_modes_non_distinct, ids=idfn) @pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn) @pytest.mark.parametrize('use_obj_hash_agg', ['false', 'true'], ids=idfn) -@allow_non_gpu(*non_utc_allow) def test_hash_groupby_collect_partial_replace_fallback(data_gen, replace_mode, aqe_enabled, @@ -857,13 +856,12 @@ def test_hash_groupby_collect_partial_replace_fallback(data_gen, @allow_non_gpu('ObjectHashAggregateExec', 'SortAggregateExec', 'ShuffleExchangeExec', 'HashPartitioning', 'SortExec', 'SortArray', 'Alias', 'Literal', 'Count', 'CollectList', 'CollectSet', - 'AggregateExpression', 'ProjectExec') + 'AggregateExpression', 'ProjectExec', *non_utc_allow) @pytest.mark.parametrize('data_gen', _full_gen_data_for_collect_op, ids=idfn) @pytest.mark.parametrize('replace_mode', _replace_modes_single_distinct, ids=idfn) @pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn) @pytest.mark.parametrize('use_obj_hash_agg', ['false', 'true'], ids=idfn) @pytest.mark.xfail(condition=is_databricks104_or_later(), reason='https://github.com/NVIDIA/spark-rapids/issues/4963') -@allow_non_gpu(*non_utc_allow) def test_hash_groupby_collect_partial_replace_with_distinct_fallback(data_gen, replace_mode, aqe_enabled, diff --git a/integration_tests/src/main/python/hive_delimited_text_test.py b/integration_tests/src/main/python/hive_delimited_text_test.py index 78c77391c31..fc2cd2e0037 100644 --- a/integration_tests/src/main/python/hive_delimited_text_test.py +++ b/integration_tests/src/main/python/hive_delimited_text_test.py @@ -121,6 +121,7 @@ def read_impl(spark): return read_impl +non_utc_allow_for_test_basic_hive_text_read=['HiveTableScanExec'] if is_not_utc() else [] @pytest.mark.skipif(is_spark_cdh(), reason="Hive text reads are disabled on CDH, as per " "https://github.com/NVIDIA/spark-rapids/pull/7628") @@ -187,7 +188,7 @@ def read_impl(spark): ('hive-delim-text/carriage-return', StructType([StructField("str", StringType())]), {}), ('hive-delim-text/carriage-return-err', StructType([StructField("str", StringType())]), {}), ], ids=idfn) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_for_test_basic_hive_text_read) def test_basic_hive_text_read(std_input_path, name, schema, spark_tmp_table_factory, options): assert_gpu_and_cpu_are_equal_collect(read_hive_text_sql(std_input_path + '/' + name, schema, spark_tmp_table_factory, options), @@ -240,7 +241,7 @@ def read_hive_text_table(spark, text_table_name, fields="my_field"): "https://github.com/NVIDIA/spark-rapids/pull/7628") @approximate_float @pytest.mark.parametrize('data_gen', hive_text_supported_gens, ids=idfn) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_for_test_basic_hive_text_read) def test_hive_text_round_trip(spark_tmp_path, data_gen, spark_tmp_table_factory): gen = StructGen([('my_field', data_gen)], nullable=False) data_path = spark_tmp_path + '/hive_text_table' @@ -527,7 +528,7 @@ def create_table_with_compressed_files(spark): ('hive-delim-text/carriage-return', StructType([StructField("str", StringType())]), {}), ('hive-delim-text/carriage-return-err', StructType([StructField("str", StringType())]), {}), ], ids=idfn) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_for_test_basic_hive_text_read) def test_basic_hive_text_write(std_input_path, input_dir, schema, spark_tmp_table_factory, mode, options): # Configure table options, including schema. if options is None: diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 116b9d6b3e1..6130e5a4acf 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -152,10 +152,12 @@ def test_json_input_meta(spark_tmp_path, v1_enabled_list): 'input_file_block_length()'), conf=updated_conf) +allow_non_gpu_for_json_scan = ['FileSourceScanExec', 'BatchScanExec'] if is_not_utc() else [] json_supported_date_formats = ['yyyy-MM-dd', 'yyyy/MM/dd', 'yyyy-MM', 'yyyy/MM', 'MM-yyyy', 'MM/yyyy', 'MM-dd-yyyy', 'MM/dd/yyyy', 'dd-MM-yyyy', 'dd/MM/yyyy'] @pytest.mark.parametrize('date_format', json_supported_date_formats, ids=idfn) @pytest.mark.parametrize('v1_enabled_list', ["", "json"]) +@allow_non_gpu(*allow_non_gpu_for_json_scan) def test_json_date_formats_round_trip(spark_tmp_path, date_format, v1_enabled_list): gen = StructGen([('a', DateGen())], nullable=False) data_path = spark_tmp_path + '/JSON_DATA' From 36cc0969930b591465ef7d79aaf6ab77337d3536 Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Wed, 29 Nov 2023 13:13:42 +0330 Subject: [PATCH 32/46] Fix more cases --- integration_tests/src/main/python/json_test.py | 6 +++--- integration_tests/src/main/python/schema_evolution_test.py | 4 +++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 6130e5a4acf..67373f2f5a2 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -183,12 +183,11 @@ def test_json_date_formats_round_trip(spark_tmp_path, date_format, v1_enabled_li "'T'HH:mm[:ss]", "'T'HH:mm"] -not_utc_allow=['BatchScanExec'] if is_not_utc() else [] - +not_utc_allow_for_test_json_scan = ['BatchScanExec', 'FileSourceScanExec'] if is_not_utc() else [] @pytest.mark.parametrize('ts_part', json_supported_ts_parts) @pytest.mark.parametrize('date_format', json_supported_date_formats) @pytest.mark.parametrize('v1_enabled_list', ["", "json"]) -@allow_non_gpu(*not_utc_allow) +@allow_non_gpu(*not_utc_allow_for_test_json_scan) def test_json_ts_formats_round_trip(spark_tmp_path, date_format, ts_part, v1_enabled_list): full_format = date_format + ts_part data_gen = TimestampGen() @@ -284,6 +283,7 @@ def do_read(spark): @pytest.mark.parametrize('allow_non_numeric_numbers', ["true", "false"]) @pytest.mark.parametrize('allow_numeric_leading_zeros', ["true"]) @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) +@allow_non_gpu(*not_utc_allow_for_test_json_scan) def test_basic_json_read(std_input_path, filename, schema, read_func, allow_non_numeric_numbers, allow_numeric_leading_zeros, ansi_enabled, spark_tmp_table_factory): updated_conf = copy_and_update(_enable_all_types_conf, {'spark.sql.ansi.enabled': ansi_enabled, diff --git a/integration_tests/src/main/python/schema_evolution_test.py b/integration_tests/src/main/python/schema_evolution_test.py index f9766a80eef..edbf1490037 100644 --- a/integration_tests/src/main/python/schema_evolution_test.py +++ b/integration_tests/src/main/python/schema_evolution_test.py @@ -16,7 +16,7 @@ from conftest import is_not_utc from data_gen import * from datetime import date, datetime, timezone -from marks import ignore_order +from marks import ignore_order, allow_non_gpu import pytest from spark_session import is_databricks_runtime, is_databricks113_or_later @@ -60,8 +60,10 @@ def get_ddl(col_gen_pairs): """Given a list of column_name, data_generator paris, returns the corresponding DDL string""" return ', '.join([f"{c} {g.data_type.simpleString()}" for c, g in col_gen_pairs]) +non_utc_allow_for_test_column_add_after_partition = ['DataWritingCommandExec'] if is_not_utc() else [] @ignore_order(local=True) @pytest.mark.parametrize("format", _formats) +@allow_non_gpu(*non_utc_allow_for_test_column_add_after_partition) def test_column_add_after_partition(spark_tmp_table_factory, format): # Databricks 10.4 appears to be missing https://issues.apache.org/jira/browse/SPARK-39417 # so avoid generating nulls for numeric partitions From 3e70b52774f48a8cb12960cbdbf763b00429c11a Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 21:50:51 +0800 Subject: [PATCH 33/46] Fix --- .../src/main/python/hive_delimited_text_test.py | 2 +- .../com/nvidia/spark/rapids/RapidsMeta.scala | 15 +++++++++++++++ .../shims/GpuCreateHiveTableAsSelectCommand.scala | 2 ++ .../rapids/shims/GpuInsertIntoHiveTable.scala | 2 ++ 4 files changed, 20 insertions(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/hive_delimited_text_test.py b/integration_tests/src/main/python/hive_delimited_text_test.py index fc2cd2e0037..eb8baa016ea 100644 --- a/integration_tests/src/main/python/hive_delimited_text_test.py +++ b/integration_tests/src/main/python/hive_delimited_text_test.py @@ -121,7 +121,7 @@ def read_impl(spark): return read_impl -non_utc_allow_for_test_basic_hive_text_read=['HiveTableScanExec'] if is_not_utc() else [] +non_utc_allow_for_test_basic_hive_text_read=['HiveTableScanExec', 'DataWritingCommandExec'] if is_not_utc() else [] @pytest.mark.skipif(is_spark_cdh(), reason="Hive text reads are disabled on CDH, as per " "https://github.com/NVIDIA/spark-rapids/pull/7628") diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 65544b1a4e4..16f0a9a2378 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.execution.command.{DataWritingCommand, RunnableComma import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} import org.apache.spark.sql.execution.python.AggregateInPandasExec +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.TimeZoneDB import org.apache.spark.sql.rapids.aggregate.{CpuToGpuAggregateBufferConverter, GpuToCpuAggregateBufferConverter} import org.apache.spark.sql.rapids.execution.{GpuBroadcastHashJoinMetaBase, GpuBroadcastNestedLoopJoinMetaBase} @@ -546,6 +547,20 @@ abstract class DataWritingCommandMeta[INPUT <: DataWritingCommand]( override val childDataWriteCmds: Seq[DataWritingCommandMeta[_]] = Seq.empty override def tagSelfForGpu(): Unit = {} + + // Check whether data type of intput/output contains time zone. + // Currently only UTC timezone is supported for [[DataWritingCommand]] + def timezoneCheck(): Unit = { + val types = (wrapped.inputSet.map(_.dataType) ++ wrapped.outputSet.map(_.dataType)).toSet + if (types.exists(GpuOverrides.isOrContainsTimestamp(_))) { + if (!GpuOverrides.isUTCTimezone()) { + willNotWorkOnGpu("Only UTC timezone is supported for " + + "GpuCreateHiveTableAsSelectCommand. " + + s"Current timezone settings: (JVM : ${ZoneId.systemDefault()}, " + + s"session: ${SQLConf.get.sessionLocalTimeZone}). ") + } + } + } } /** diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuCreateHiveTableAsSelectCommand.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuCreateHiveTableAsSelectCommand.scala index 5d9e8e97bed..253537038e7 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuCreateHiveTableAsSelectCommand.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuCreateHiveTableAsSelectCommand.scala @@ -88,6 +88,8 @@ final class GpuCreateHiveTableAsSelectCommandMeta(cmd: CreateHiveTableAsSelectCo }) } } + + timezoneCheck() } override def convertToGpu(): GpuDataWritingCommand = GpuCreateHiveTableAsSelectCommand( diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala index 2e10cda8414..ba4ff6d7a46 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala @@ -72,6 +72,8 @@ final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, // Only Hive delimited text writes are currently supported. // Check whether that is the format currently in play. fileFormat = GpuHiveTextFileFormat.tagGpuSupport(this) + + timezoneCheck() } override def convertToGpu(): GpuDataWritingCommand = { From 51e501758f3c609226645d227fdb2dc5d70d5f1f Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 22:39:53 +0800 Subject: [PATCH 34/46] Refactor --- .../nvidia/spark/rapids/GpuOverrides.scala | 2 +- .../com/nvidia/spark/rapids/RapidsMeta.scala | 20 +++++++++++++++---- ...aSourceTableAsSelectCommandMetaShims.scala | 2 +- ...dCreateHiveTableAsSelectCommandShims.scala | 2 +- .../GpuCreateHiveTableAsSelectCommand.scala | 4 +--- .../rapids/shims/GpuInsertIntoHiveTable.scala | 4 +--- .../rapids/shims/GpuInsertIntoHiveTable.scala | 2 +- 7 files changed, 22 insertions(+), 14 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 9d5c58cd628..a3b407307db 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -319,7 +319,7 @@ final class InsertIntoHadoopFsRelationCommandMeta( private var fileFormat: Option[ColumnarFileFormat] = None - override def tagSelfForGpu(): Unit = { + override def tagSelfForGpuInternal(): Unit = { if (cmd.bucketSpec.isDefined) { willNotWorkOnGpu("bucketing is not supported") } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 16f0a9a2378..32c5070fede 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -546,10 +546,22 @@ abstract class DataWritingCommandMeta[INPUT <: DataWritingCommand]( override val childParts: Seq[PartMeta[_]] = Seq.empty override val childDataWriteCmds: Seq[DataWritingCommandMeta[_]] = Seq.empty - override def tagSelfForGpu(): Unit = {} + val checkTimeZone: Boolean = true - // Check whether data type of intput/output contains time zone. - // Currently only UTC timezone is supported for [[DataWritingCommand]] + final override def tagSelfForGpu(): Unit = { + if (checkTimeZone) { + timezoneCheck() + } + tagSelfForGpuInternal() + } + + protected def tagSelfForGpuInternal(): Unit = {} + + // Check whether data type of intput/output contains timestamp type, which + // is related to time zone. + // Only UTC time zone is allowed to be consistent with previous behavior + // for [[DataWritingCommand]]. Needs to override [[checkTimeZone]] to skip + // UTC time zone check in sub class of [[DataWritingCommand]]. def timezoneCheck(): Unit = { val types = (wrapped.inputSet.map(_.dataType) ++ wrapped.outputSet.map(_.dataType)).toSet if (types.exists(GpuOverrides.isOrContainsTimestamp(_))) { @@ -572,7 +584,7 @@ final class RuleNotFoundDataWritingCommandMeta[INPUT <: DataWritingCommand]( parent: Option[RapidsMeta[_, _, _]]) extends DataWritingCommandMeta[INPUT](cmd, conf, parent, new NoRuleDataFromReplacementRule) { - override def tagSelfForGpu(): Unit = { + override def tagSelfForGpuInternal(): Unit = { willNotWorkOnGpu(s"GPU does not currently support the operator ${cmd.getClass}") } diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala index 6d515d4845d..42fe88916c6 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala @@ -55,7 +55,7 @@ final class CreateDataSourceTableAsSelectCommandMeta( private var origProvider: Class[_] = _ private var gpuProvider: Option[ColumnarFileFormat] = None - override def tagSelfForGpu(): Unit = { + override def tagSelfForGpuInternal(): Unit = { if (cmd.table.bucketSpec.isDefined) { willNotWorkOnGpu("bucketing is not supported") } diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala index afcec8d5706..869b4a2b3b8 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala @@ -171,7 +171,7 @@ final class OptimizedCreateHiveTableAsSelectCommandMeta( extends DataWritingCommandMeta[OptimizedCreateHiveTableAsSelectCommand]( cmd, conf, parent, rule) { - override def tagSelfForGpu(): Unit = { + override def tagSelfForGpuInternal(): Unit = { // It would be cleaner if we could simply call `cmd.getWritingCommand` and let // InsertIntoHadoopFsRelationCommandMeta tag the result, but calling getWritingCommand // before the table exists will crash. So this ends up replicating a portion of the logic diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuCreateHiveTableAsSelectCommand.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuCreateHiveTableAsSelectCommand.scala index 253537038e7..2e7a03b8def 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuCreateHiveTableAsSelectCommand.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuCreateHiveTableAsSelectCommand.scala @@ -52,7 +52,7 @@ final class GpuCreateHiveTableAsSelectCommandMeta(cmd: CreateHiveTableAsSelectCo private var cpuWritingCommand: Option[InsertIntoHiveTable] = None - override def tagSelfForGpu(): Unit = { + override def tagSelfForGpuInternal(): Unit = { val spark = SparkSession.active val tableDesc = cmd.tableDesc // For the *new* table. @@ -88,8 +88,6 @@ final class GpuCreateHiveTableAsSelectCommandMeta(cmd: CreateHiveTableAsSelectCo }) } } - - timezoneCheck() } override def convertToGpu(): GpuDataWritingCommand = GpuCreateHiveTableAsSelectCommand( diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala index ba4ff6d7a46..a4e010bbc7b 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala @@ -68,12 +68,10 @@ final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, private var fileFormat: Option[ColumnarFileFormat] = None - override def tagSelfForGpu(): Unit = { + override def tagSelfForGpuInternal(): Unit = { // Only Hive delimited text writes are currently supported. // Check whether that is the format currently in play. fileFormat = GpuHiveTextFileFormat.tagGpuSupport(this) - - timezoneCheck() } override def convertToGpu(): GpuDataWritingCommand = { diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala index c4f7235639b..cc4b005a1ec 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala @@ -54,7 +54,7 @@ final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, private var fileFormat: Option[ColumnarFileFormat] = None - override def tagSelfForGpu(): Unit = { + override def tagSelfForGpuInternal(): Unit = { // Only Hive delimited text writes are currently supported. // Check whether that is the format currently in play. fileFormat = GpuHiveTextFileFormat.tagGpuSupport(this) From 07819fb0e9db9c2f4f7791f4a06ea6abc8a7679a Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Wed, 29 Nov 2023 17:39:18 +0330 Subject: [PATCH 35/46] Fix more cases 3 --- integration_tests/src/main/python/cmp_test.py | 3 +++ integration_tests/src/main/python/data_gen.py | 2 +- .../src/main/python/hive_delimited_text_test.py | 6 +++--- integration_tests/src/main/python/orc_cast_test.py | 6 +++--- integration_tests/src/main/python/schema_evolution_test.py | 2 +- 5 files changed, 11 insertions(+), 8 deletions(-) diff --git a/integration_tests/src/main/python/cmp_test.py b/integration_tests/src/main/python/cmp_test.py index 8a92c268182..59759098f28 100644 --- a/integration_tests/src/main/python/cmp_test.py +++ b/integration_tests/src/main/python/cmp_test.py @@ -291,11 +291,14 @@ def test_filter_with_project(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, BooleanGen(), data_gen).filter(f.col('a')).selectExpr('*', 'a as a2')) +# DateAddInterval is a time zone aware expression +non_utc_allow_for_date_add_interval = ['ProjectExec', 'FilterExec'] if is_not_utc() else [] # It takes quite a bit to get filter to have a column it can filter on, but # no columns to actually filter. We are making it happen here with a sub-query # and some constants that then make it so all we need is the number of rows # of input. @pytest.mark.parametrize('op', ['>', '<']) +@allow_non_gpu(*non_utc_allow_for_date_add_interval) def test_empty_filter(op, spark_tmp_path): def do_it(spark): diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index da81309c69e..ac78a29dfa9 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -1175,4 +1175,4 @@ def get_25_partitions_df(spark): # allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' -non_utc_allow=['ProjectExec', 'FilterExec', 'FileSourceScanExec', 'BatchScanExec', 'CollectLimitExec', 'DeserializeToObjectExec', 'DataWritingCommandExec', 'WriteFilesExec', 'ShuffleExchangeExec'] if is_not_utc() else [] \ No newline at end of file +non_utc_allow = ['ProjectExec', 'FilterExec', 'FileSourceScanExec', 'BatchScanExec', 'CollectLimitExec', 'DeserializeToObjectExec', 'DataWritingCommandExec', 'WriteFilesExec', 'ShuffleExchangeExec'] if is_not_utc() else [] \ No newline at end of file diff --git a/integration_tests/src/main/python/hive_delimited_text_test.py b/integration_tests/src/main/python/hive_delimited_text_test.py index eb8baa016ea..b0467aa8910 100644 --- a/integration_tests/src/main/python/hive_delimited_text_test.py +++ b/integration_tests/src/main/python/hive_delimited_text_test.py @@ -121,7 +121,7 @@ def read_impl(spark): return read_impl -non_utc_allow_for_test_basic_hive_text_read=['HiveTableScanExec', 'DataWritingCommandExec'] if is_not_utc() else [] +non_utc_allow_for_test_basic_hive_text_read=['HiveTableScanExec', 'DataWritingCommandExec', 'WriteFilesExec'] if is_not_utc() else [] @pytest.mark.skipif(is_spark_cdh(), reason="Hive text reads are disabled on CDH, as per " "https://github.com/NVIDIA/spark-rapids/pull/7628") @@ -283,7 +283,7 @@ def read_hive_text_table_partitions(spark, text_table_name, partition): reason="Hive text reads are disabled on CDH, as per " "https://github.com/NVIDIA/spark-rapids/pull/7628") @approximate_float -@allow_non_gpu("EqualTo,IsNotNull,Literal", *non_utc_allow) # Accounts for partition predicate: `WHERE dt='1'` +@allow_non_gpu("EqualTo,IsNotNull,Literal", *non_utc_allow_for_test_basic_hive_text_read) # Accounts for partition predicate: `WHERE dt='1'` @pytest.mark.parametrize('data_gen', hive_text_supported_gens, ids=idfn) def test_hive_text_round_trip_partitioned(spark_tmp_path, data_gen, spark_tmp_table_factory): gen = StructGen([('my_field', data_gen)], nullable=False) @@ -301,7 +301,7 @@ def test_hive_text_round_trip_partitioned(spark_tmp_path, data_gen, spark_tmp_ta reason="Hive text reads are disabled on CDH, as per " "https://github.com/NVIDIA/spark-rapids/pull/7628") @approximate_float -@allow_non_gpu("EqualTo,IsNotNull,Literal,Or", *non_utc_allow) # Accounts for partition predicate +@allow_non_gpu("EqualTo,IsNotNull,Literal,Or", *non_utc_allow_for_test_basic_hive_text_read) # Accounts for partition predicate @pytest.mark.parametrize('data_gen', hive_text_supported_gens, ids=idfn) def test_hive_text_round_trip_two_partitions(spark_tmp_path, data_gen, spark_tmp_table_factory): """ diff --git a/integration_tests/src/main/python/orc_cast_test.py b/integration_tests/src/main/python/orc_cast_test.py index f358c2f6207..48efd5c8174 100644 --- a/integration_tests/src/main/python/orc_cast_test.py +++ b/integration_tests/src/main/python/orc_cast_test.py @@ -70,11 +70,11 @@ def test_casting_from_integer(spark_tmp_path, to_type): schema_str.format(*([to_type] * len(data_gen)))).orc(orc_path) ) - +non_utc_allow_for_test_casting_from_overflow_long = ['FileSourceScanExec', 'ColumnarToRowExec', ] if is_not_utc() else [] @pytest.mark.parametrize('overflow_long_gen', [LongGen(min_val=int(1e16)), LongGen(max_val=int(-1e16))]) @pytest.mark.parametrize('to_type', ['timestamp']) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_for_test_casting_from_overflow_long) def test_casting_from_overflow_long(spark_tmp_path, overflow_long_gen,to_type): # Timestamp(micro-seconds) is actually type of int64, when casting long(int64) to timestamp, # we need to multiply 1e6 (or 1e3), and it may cause overflow. This function aims to test @@ -129,7 +129,7 @@ def test_casting_from_double_to_timestamp(spark_tmp_path, data_gen): ) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow_for_test_casting_from_overflow_long) def test_casting_from_overflow_double_to_timestamp(spark_tmp_path): orc_path = spark_tmp_path + '/orc_casting_from_overflow_double_to_timestamp' with_cpu_session( diff --git a/integration_tests/src/main/python/schema_evolution_test.py b/integration_tests/src/main/python/schema_evolution_test.py index edbf1490037..162a3671557 100644 --- a/integration_tests/src/main/python/schema_evolution_test.py +++ b/integration_tests/src/main/python/schema_evolution_test.py @@ -60,7 +60,7 @@ def get_ddl(col_gen_pairs): """Given a list of column_name, data_generator paris, returns the corresponding DDL string""" return ', '.join([f"{c} {g.data_type.simpleString()}" for c, g in col_gen_pairs]) -non_utc_allow_for_test_column_add_after_partition = ['DataWritingCommandExec'] if is_not_utc() else [] +non_utc_allow_for_test_column_add_after_partition = ['DataWritingCommandExec', 'FileSourceScanExec'] if is_not_utc() else [] @ignore_order(local=True) @pytest.mark.parametrize("format", _formats) @allow_non_gpu(*non_utc_allow_for_test_column_add_after_partition) From d9006073d0647802fc10238be1d50c0d61c31b8a Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 23:10:58 +0800 Subject: [PATCH 36/46] Address comments --- integration_tests/src/main/python/aqe_test.py | 4 ++-- .../src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/integration_tests/src/main/python/aqe_test.py b/integration_tests/src/main/python/aqe_test.py index 1ad8f4f5cb0..06759954631 100755 --- a/integration_tests/src/main/python/aqe_test.py +++ b/integration_tests/src/main/python/aqe_test.py @@ -22,7 +22,7 @@ from spark_session import with_cpu_session, is_databricks113_or_later # allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' -non_utc_allow=['HashAggregateExec', 'ProjectExec', 'FilterExec', 'FileSourceScanExec', 'BatchScanExec', 'CollectLimitExec', 'DeserializeToObjectExec', 'DataWritingCommandExec', 'WriteFilesExec', 'ShuffleExchangeExec'] if is_not_utc() else [] +not_utc_aqe_allow=['ShuffleExchangeExec', 'HashAggregateExec'] if is_not_utc() else [] _adaptive_conf = { "spark.sql.adaptive.enabled": "true" } @@ -197,7 +197,7 @@ def do_it(spark): # broadcast join. The bug currently manifests in Databricks, but could # theoretically show up in other Spark distributions @ignore_order(local=True) -@allow_non_gpu('BroadcastNestedLoopJoinExec', 'Cast', 'DateSub', *db_113_cpu_bnlj_join_allow, *non_utc_allow) +@allow_non_gpu('BroadcastNestedLoopJoinExec', 'Cast', 'DateSub', *db_113_cpu_bnlj_join_allow, *not_utc_aqe_allow) @pytest.mark.parametrize('join', joins, ids=idfn) def test_aqe_join_reused_exchange_inequality_condition(spark_tmp_path, join): data_path = spark_tmp_path + '/PARQUET_DATA' diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 32c5070fede..15c22bce6cd 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -567,7 +567,7 @@ abstract class DataWritingCommandMeta[INPUT <: DataWritingCommand]( if (types.exists(GpuOverrides.isOrContainsTimestamp(_))) { if (!GpuOverrides.isUTCTimezone()) { willNotWorkOnGpu("Only UTC timezone is supported for " + - "GpuCreateHiveTableAsSelectCommand. " + + s"${wrapped.getClass}. " + s"Current timezone settings: (JVM : ${ZoneId.systemDefault()}, " + s"session: ${SQLConf.get.sessionLocalTimeZone}). ") } From 6e38bcb4dffacb95f28ed18db5aa8fe7df2c16d5 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Wed, 29 Nov 2023 23:18:26 +0800 Subject: [PATCH 37/46] Address comments --- .../src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 15c22bce6cd..d9496ff793b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -566,8 +566,7 @@ abstract class DataWritingCommandMeta[INPUT <: DataWritingCommand]( val types = (wrapped.inputSet.map(_.dataType) ++ wrapped.outputSet.map(_.dataType)).toSet if (types.exists(GpuOverrides.isOrContainsTimestamp(_))) { if (!GpuOverrides.isUTCTimezone()) { - willNotWorkOnGpu("Only UTC timezone is supported for " + - s"${wrapped.getClass}. " + + willNotWorkOnGpu("Only UTC timezone is supported. " + s"Current timezone settings: (JVM : ${ZoneId.systemDefault()}, " + s"session: ${SQLConf.get.sessionLocalTimeZone}). ") } From dd07316574a2df85abadd58d2fcd2bc82a9d51d7 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Thu, 30 Nov 2023 08:07:17 +0800 Subject: [PATCH 38/46] Fix for 341 Fix for 341 --- integration_tests/src/main/python/json_test.py | 11 ++++++----- integration_tests/src/main/python/orc_write_test.py | 4 +++- .../src/main/python/parquet_write_test.py | 4 +++- .../src/main/python/schema_evolution_test.py | 2 +- 4 files changed, 13 insertions(+), 8 deletions(-) diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index e5929d5da4f..0583477a729 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -206,20 +206,21 @@ def test_json_ts_formats_round_trip(spark_tmp_path, date_format, ts_part, v1_ena .json(data_path), conf=updated_conf) -@allow_non_gpu('FileSourceScanExec', 'ProjectExec', *non_utc_allow) -@pytest.mark.skipif(is_before_spark_341(), reason='`TIMESTAMP_NTZ` is only supported in PySpark 341+') +@allow_non_gpu('FileSourceScanExec', 'ProjectExec') +@pytest.mark.skipif(is_before_spark_341(), reason='`TIMESTAMP_NTZ` is only supported in PySpark 341+.') +@pytest.mark.xfail(is_not_utc(), reason='Timezone is not supported for json format as https://github.com/NVIDIA/spark-rapids/issues/9653.') @pytest.mark.parametrize('ts_part', json_supported_ts_parts) @pytest.mark.parametrize('date_format', json_supported_date_formats) @pytest.mark.parametrize("timestamp_type", ["TIMESTAMP_LTZ", "TIMESTAMP_NTZ"]) def test_json_ts_formats_round_trip_ntz_v1(spark_tmp_path, date_format, ts_part, timestamp_type): json_ts_formats_round_trip_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, 'json', 'FileSourceScanExec') -@allow_non_gpu('BatchScanExec', 'ProjectExec', *non_utc_allow) -@pytest.mark.skipif(is_before_spark_341(), reason='`TIMESTAMP_NTZ` is only supported in PySpark 341+') +@allow_non_gpu('BatchScanExec', 'ProjectExec') +@pytest.mark.skipif(is_before_spark_341(), reason='`TIMESTAMP_NTZ` is only supported in PySpark 341+.') +@pytest.mark.xfail(is_not_utc(), reason='Timezone is not supported for json format as https://github.com/NVIDIA/spark-rapids/issues/9653.') @pytest.mark.parametrize('ts_part', json_supported_ts_parts) @pytest.mark.parametrize('date_format', json_supported_date_formats) @pytest.mark.parametrize("timestamp_type", ["TIMESTAMP_LTZ", "TIMESTAMP_NTZ"]) - def test_json_ts_formats_round_trip_ntz_v2(spark_tmp_path, date_format, ts_part, timestamp_type): json_ts_formats_round_trip_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, '', 'BatchScanExec') diff --git a/integration_tests/src/main/python/orc_write_test.py b/integration_tests/src/main/python/orc_write_test.py index c6244921677..a28aad58406 100644 --- a/integration_tests/src/main/python/orc_write_test.py +++ b/integration_tests/src/main/python/orc_write_test.py @@ -189,11 +189,13 @@ def write_orc_sql_from(spark, df, data_path, write_to_table): write_cmd = 'CREATE TABLE `{}` USING ORC location \'{}\' AS SELECT * from `{}`'.format(write_to_table, data_path, tmp_view_name) spark.sql(write_cmd) +non_utc_hive_save_table_allow = ['ExecutedCommandExec', 'DataWritingCommandExec', 'CreateDataSourceTableAsSelectCommand', 'WriteFilesExec'] if is_not_utc() else [] + @pytest.mark.order(2) @pytest.mark.parametrize('orc_gens', orc_write_gens_list, ids=idfn) @pytest.mark.parametrize('ts_type', ["TIMESTAMP_MICROS", "TIMESTAMP_MILLIS"]) @pytest.mark.parametrize('orc_impl', ["native", "hive"]) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_hive_save_table_allow) def test_write_sql_save_table(spark_tmp_path, orc_gens, ts_type, orc_impl, spark_tmp_table_factory): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)] data_path = spark_tmp_path + '/ORC_DATA' diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index e25546fcd00..75fa9034895 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -265,9 +265,11 @@ def write_parquet_sql_from(spark, df, data_path, write_to_table): write_cmd = 'CREATE TABLE `{}` USING PARQUET location \'{}\' AS SELECT * from `{}`'.format(write_to_table, data_path, tmp_view_name) spark.sql(write_cmd) +non_utc_hive_save_table_allow = ['ExecutedCommandExec', 'DataWritingCommandExec', 'CreateDataSourceTableAsSelectCommand', 'WriteFilesExec'] if is_not_utc() else [] + @pytest.mark.order(2) @pytest.mark.parametrize('parquet_gens', parquet_write_gens_list, ids=idfn) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_hive_save_table_allow) def test_write_sql_save_table(spark_tmp_path, parquet_gens, spark_tmp_table_factory): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] data_path = spark_tmp_path + '/PARQUET_DATA' diff --git a/integration_tests/src/main/python/schema_evolution_test.py b/integration_tests/src/main/python/schema_evolution_test.py index 162a3671557..ff501324cc0 100644 --- a/integration_tests/src/main/python/schema_evolution_test.py +++ b/integration_tests/src/main/python/schema_evolution_test.py @@ -60,7 +60,7 @@ def get_ddl(col_gen_pairs): """Given a list of column_name, data_generator paris, returns the corresponding DDL string""" return ', '.join([f"{c} {g.data_type.simpleString()}" for c, g in col_gen_pairs]) -non_utc_allow_for_test_column_add_after_partition = ['DataWritingCommandExec', 'FileSourceScanExec'] if is_not_utc() else [] +non_utc_allow_for_test_column_add_after_partition = ['ColumnarToRowExec', 'DataWritingCommandExec', 'ExecutedCommandExec', 'FileSourceScanExec', 'WriteFilesExec'] if is_not_utc() else [] @ignore_order(local=True) @pytest.mark.parametrize("format", _formats) @allow_non_gpu(*non_utc_allow_for_test_column_add_after_partition) From 2e6578e792f362b1a7c64f11666eb58ca82b6fd7 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Thu, 30 Nov 2023 13:48:19 +0800 Subject: [PATCH 39/46] Fix 341 --- integration_tests/src/main/python/cache_test.py | 5 ++++- integration_tests/src/main/python/csv_test.py | 7 ++++--- integration_tests/src/main/python/data_gen.py | 5 ++++- integration_tests/src/main/python/json_test.py | 1 + integration_tests/src/main/python/parquet_write_test.py | 4 +++- 5 files changed, 16 insertions(+), 6 deletions(-) diff --git a/integration_tests/src/main/python/cache_test.py b/integration_tests/src/main/python/cache_test.py index d2318a50c18..70fb95fc1ea 100644 --- a/integration_tests/src/main/python/cache_test.py +++ b/integration_tests/src/main/python/cache_test.py @@ -151,6 +151,9 @@ def n_fold(spark): with_cpu_session(n_fold) +# allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' +non_utc_orc_save_table_allow = ['DataWritingCommandExec', 'WriteFilesExec'] if is_not_utc() else [] + # This test doesn't allow negative scale for Decimals as ` df.write.mode('overwrite').parquet(data_path)` # writes parquet which doesn't allow negative decimals @pytest.mark.parametrize('data_gen', [StringGen(), ByteGen(), ShortGen(), IntegerGen(), LongGen(), @@ -165,7 +168,7 @@ def n_fold(spark): @pytest.mark.parametrize('ts_write', ['TIMESTAMP_MICROS', 'TIMESTAMP_MILLIS']) @pytest.mark.parametrize('enable_vectorized', ['true', 'false'], ids=idfn) @ignore_order -@allow_non_gpu("SortExec", "ShuffleExchangeExec", "RangePartitioning") +@allow_non_gpu("SortExec", "ShuffleExchangeExec", "RangePartitioning", *non_utc_orc_save_table_allow) def test_cache_columnar(spark_tmp_path, data_gen, enable_vectorized, ts_write): data_path_gpu = spark_tmp_path + '/PARQUET_DATA' def read_parquet_cached(data_path): diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index d7836d77c5b..0dca1d60b55 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -565,13 +565,14 @@ def test_csv_read_count(spark_tmp_path): assert_gpu_and_cpu_row_counts_equal(lambda spark: spark.read.csv(data_path), conf = {'spark.rapids.sql.explain': 'ALL'}) -@allow_non_gpu('FileSourceScanExec', 'ProjectExec', 'CollectLimitExec', 'DeserializeToObjectExec', *non_utc_allow) +@allow_non_gpu('FileSourceScanExec', 'ProjectExec', 'CollectLimitExec', 'DeserializeToObjectExec') @pytest.mark.skipif(is_before_spark_341(), reason='`TIMESTAMP_NTZ` is only supported in PySpark 341+') @pytest.mark.parametrize('date_format', csv_supported_date_formats) @pytest.mark.parametrize('ts_part', csv_supported_ts_parts) @pytest.mark.parametrize("timestamp_type", [ pytest.param('TIMESTAMP_LTZ', marks=pytest.mark.xfail(is_spark_350_or_later(), reason="https://github.com/NVIDIA/spark-rapids/issues/9325")), "TIMESTAMP_NTZ"]) +@pytest.mark.xfail(is_not_utc(), reason='Timezone is not supported for csv format as https://github.com/NVIDIA/spark-rapids/issues/9653.') def test_csv_infer_schema_timestamp_ntz_v1(spark_tmp_path, date_format, ts_part, timestamp_type): csv_infer_schema_timestamp_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, 'csv', 'FileSourceScanExec') @@ -622,9 +623,9 @@ def do_read(spark): non_exist_classes = cpu_scan_class, conf = conf) -@allow_non_gpu('FileSourceScanExec', 'CollectLimitExec', 'DeserializeToObjectExec', *non_utc_allow) +@allow_non_gpu('FileSourceScanExec', 'CollectLimitExec', 'DeserializeToObjectExec') @pytest.mark.skipif(is_before_spark_340(), reason='`preferDate` is only supported in Spark 340+') - +@pytest.mark.xfail(is_not_utc(), reason='Timezone is not supported for csv format as https://github.com/NVIDIA/spark-rapids/issues/9653.') def test_csv_prefer_date_with_infer_schema(spark_tmp_path): # start date ""0001-01-02" required due to: https://github.com/NVIDIA/spark-rapids/issues/5606 data_gens = [byte_gen, short_gen, int_gen, long_gen, boolean_gen, timestamp_gen, DateGen(start=date(1, 1, 2))] diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index ac78a29dfa9..e82dde3eeaf 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -1175,4 +1175,7 @@ def get_25_partitions_df(spark): # allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' -non_utc_allow = ['ProjectExec', 'FilterExec', 'FileSourceScanExec', 'BatchScanExec', 'CollectLimitExec', 'DeserializeToObjectExec', 'DataWritingCommandExec', 'WriteFilesExec', 'ShuffleExchangeExec'] if is_not_utc() else [] \ No newline at end of file +# This will be deprecated and replaced case specified non GPU allow list +non_utc_allow = ['ProjectExec', 'FilterExec', 'FileSourceScanExec', 'BatchScanExec', 'CollectLimitExec', + 'DeserializeToObjectExec', 'DataWritingCommandExec', 'WriteFilesExec', 'ShuffleExchangeExec', + 'ExecutedCommandExec'] if is_not_utc() else [] \ No newline at end of file diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 0583477a729..d1f34826d65 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -853,6 +853,7 @@ def struct_to_json(spark): 'Etc/UTC', pytest.param('UTC+07:00', marks=pytest.mark.allow_non_gpu('ProjectExec')), ]) +@pytest.mark.skipif(is_not_utc(), reason='Duplicated as original test case designed which it is parameterized by timezone. https://github.com/NVIDIA/spark-rapids/issues/9653.') def test_structs_to_json_timestamp(spark_tmp_path, data_gen, timestamp_format, timezone): struct_gen = StructGen([ ("b", StructGen([('child', data_gen)], nullable=True)), diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index 75fa9034895..d07def7483f 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -771,10 +771,12 @@ def read_table(spark, path): func(create_table, read_table, data_path, conf) +non_utc_hive_parquet_write_allow = ['DataWritingCommandExec', 'WriteFilesExec'] if is_not_utc() else [] + # Test to avoid regression on a known bug in Spark. For details please visit https://github.com/NVIDIA/spark-rapids/issues/8693 @pytest.mark.parametrize('ts_rebase', [ pytest.param('LEGACY', marks=pytest.mark.skipif(is_not_utc(), reason="LEGACY datetime rebase mode is only supported for UTC timezone")), - 'CORRECTED']) + pytest.param('CORRECTED', marks=pytest.mark.allow_non_gpu(*non_utc_hive_parquet_write_allow))]) def test_hive_timestamp_value(spark_tmp_table_factory, spark_tmp_path, ts_rebase): def func_test(create_table, read_table, data_path, conf): assert_gpu_and_cpu_writes_are_equal_collect(create_table, read_table, data_path, conf=conf) From 33187b0904e63fb85c7306f792f38f79d45ced20 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Thu, 30 Nov 2023 16:27:11 +0800 Subject: [PATCH 40/46] Minor fix --- integration_tests/src/main/python/arithmetic_ops_test.py | 1 - integration_tests/src/main/python/ast_test.py | 1 - integration_tests/src/main/python/collection_ops_test.py | 1 - 3 files changed, 3 deletions(-) diff --git a/integration_tests/src/main/python/arithmetic_ops_test.py b/integration_tests/src/main/python/arithmetic_ops_test.py index e9fbfdf0c3e..b75872ed8b2 100644 --- a/integration_tests/src/main/python/arithmetic_ops_test.py +++ b/integration_tests/src/main/python/arithmetic_ops_test.py @@ -16,7 +16,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error, assert_gpu_fallback_collect, assert_gpu_and_cpu_are_equal_sql -from conftest import is_not_utc from data_gen import * from marks import ignore_order, incompat, approximate_float, allow_non_gpu, datagen_overrides from pyspark.sql.types import * diff --git a/integration_tests/src/main/python/ast_test.py b/integration_tests/src/main/python/ast_test.py index a7af84180c5..21e982e4fb8 100644 --- a/integration_tests/src/main/python/ast_test.py +++ b/integration_tests/src/main/python/ast_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_cpu_and_gpu_are_equal_collect_with_capture -from conftest import is_not_utc from data_gen import * from marks import approximate_float, datagen_overrides from spark_session import with_cpu_session, is_before_spark_330 diff --git a/integration_tests/src/main/python/collection_ops_test.py b/integration_tests/src/main/python/collection_ops_test.py index 87fde6f9ac1..528d57cf722 100644 --- a/integration_tests/src/main/python/collection_ops_test.py +++ b/integration_tests/src/main/python/collection_ops_test.py @@ -25,7 +25,6 @@ from conftest import get_datagen_seed from marks import allow_non_gpu - nested_gens = [ArrayGen(LongGen()), ArrayGen(decimal_gen_128bit), StructGen([("a", LongGen()), ("b", decimal_gen_128bit)]), MapGen(StringGen(pattern='key_[0-9]', nullable=False), StringGen()), From 6dff012eddf5ee628485b1df50bf3b2eabd35bf5 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Thu, 30 Nov 2023 18:07:50 +0800 Subject: [PATCH 41/46] Enable golden configuration --- integration_tests/src/main/python/collection_ops_test.py | 1 - integration_tests/src/main/python/conditionals_test.py | 1 - integration_tests/src/main/python/csv_test.py | 5 +++-- integration_tests/src/main/python/datasourcev2_read_test.py | 3 +-- integration_tests/src/main/python/expand_exec_test.py | 1 - .../src/main/python/fastparquet_compatibility_test.py | 1 - integration_tests/src/main/python/generate_expr_test.py | 1 - integration_tests/src/main/python/hashing_test.py | 1 - integration_tests/src/main/python/json_test.py | 1 - integration_tests/src/main/python/limit_test.py | 1 - integration_tests/src/main/python/row-based_udf_test.py | 1 - integration_tests/src/main/python/row_conversion_test.py | 1 - integration_tests/src/main/python/sample_test.py | 1 - integration_tests/src/main/python/struct_test.py | 1 - integration_tests/src/main/python/subquery_test.py | 1 - integration_tests/src/main/python/time_window_test.py | 1 - integration_tests/src/main/python/udf_test.py | 2 +- integration_tests/src/main/python/window_function_test.py | 1 - .../src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala | 5 +++-- 19 files changed, 8 insertions(+), 22 deletions(-) diff --git a/integration_tests/src/main/python/collection_ops_test.py b/integration_tests/src/main/python/collection_ops_test.py index 528d57cf722..16a450a1a8c 100644 --- a/integration_tests/src/main/python/collection_ops_test.py +++ b/integration_tests/src/main/python/collection_ops_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error -from conftest import is_not_utc from data_gen import * from pyspark.sql.types import * from string_test import mk_str_gen diff --git a/integration_tests/src/main/python/conditionals_test.py b/integration_tests/src/main/python/conditionals_test.py index b418483fb10..bc66c6e931c 100644 --- a/integration_tests/src/main/python/conditionals_test.py +++ b/integration_tests/src/main/python/conditionals_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect -from conftest import is_not_utc from data_gen import * from spark_session import is_before_spark_320, is_jvm_charset_utf8 from pyspark.sql.types import * diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index 0dca1d60b55..9b9c51f06fa 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -334,7 +334,7 @@ def test_csv_fallback(spark_tmp_path, read_func, disable_conf, spark_tmp_table_f 'CORRECTED', 'EXCEPTION' ]) -@allow_non_gpu(*non_utc_allow) +@allow_non_gpu(*non_utc_allow) # Date is also time zone related for csv since rebasing. def test_date_formats_round_trip(spark_tmp_path, date_format, v1_enabled_list, ansi_enabled, time_parser_policy): gen = StructGen([('a', DateGen())], nullable=False) data_path = spark_tmp_path + '/CSV_DATA' @@ -366,13 +366,14 @@ def test_date_formats_round_trip(spark_tmp_path, date_format, v1_enabled_list, a .csv(data_path), conf=updated_conf) +non_utc_allow_for_test_read_valid_and_invalid_dates=['BatchScanExec', 'FileSourceScanExec'] if is_not_utc() else [] -non_utc_allow_for_test_read_valid_and_invalid_dates=['FileSourceScanExec', 'BatchScanExec'] if is_not_utc() else [] @pytest.mark.parametrize('filename', ["date.csv"]) @pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) @pytest.mark.parametrize('time_parser_policy', [ pytest.param('LEGACY', marks=pytest.mark.allow_non_gpu('BatchScanExec,FileSourceScanExec')), + # Date is also time zone related for csv since rebasing. pytest.param('CORRECTED', marks=pytest.mark.allow_non_gpu(*non_utc_allow_for_test_read_valid_and_invalid_dates)), pytest.param('EXCEPTION', marks=pytest.mark.allow_non_gpu(*non_utc_allow_for_test_read_valid_and_invalid_dates)) ]) diff --git a/integration_tests/src/main/python/datasourcev2_read_test.py b/integration_tests/src/main/python/datasourcev2_read_test.py index b2134c50618..a3414f5c32a 100644 --- a/integration_tests/src/main/python/datasourcev2_read_test.py +++ b/integration_tests/src/main/python/datasourcev2_read_test.py @@ -15,8 +15,7 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_row_counts_equal -from conftest import is_not_utc -from data_gen import * +from data_gen import non_utc_allow from marks import * columnarClass = 'com.nvidia.spark.rapids.tests.datasourcev2.parquet.ArrowColumnarDataSourceV2' diff --git a/integration_tests/src/main/python/expand_exec_test.py b/integration_tests/src/main/python/expand_exec_test.py index 5a8be470a44..d53000e9849 100644 --- a/integration_tests/src/main/python/expand_exec_test.py +++ b/integration_tests/src/main/python/expand_exec_test.py @@ -14,7 +14,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect, assert_equal -from conftest import is_not_utc from data_gen import * import pyspark.sql.functions as f from marks import ignore_order diff --git a/integration_tests/src/main/python/fastparquet_compatibility_test.py b/integration_tests/src/main/python/fastparquet_compatibility_test.py index 86cac2aefed..b51fa5a55ef 100644 --- a/integration_tests/src/main/python/fastparquet_compatibility_test.py +++ b/integration_tests/src/main/python/fastparquet_compatibility_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect -from conftest import is_not_utc from data_gen import * from fastparquet_utils import get_fastparquet_result_canonicalizer from spark_session import is_databricks_runtime, spark_version, with_cpu_session, with_gpu_session diff --git a/integration_tests/src/main/python/generate_expr_test.py b/integration_tests/src/main/python/generate_expr_test.py index 7e8c5ced399..66c56ebcd19 100644 --- a/integration_tests/src/main/python/generate_expr_test.py +++ b/integration_tests/src/main/python/generate_expr_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_fallback_collect -from conftest import is_not_utc from data_gen import * from marks import allow_non_gpu, ignore_order from pyspark.sql.types import * diff --git a/integration_tests/src/main/python/hashing_test.py b/integration_tests/src/main/python/hashing_test.py index 93a7f30e756..6bd56da933d 100644 --- a/integration_tests/src/main/python/hashing_test.py +++ b/integration_tests/src/main/python/hashing_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_fallback_collect -from conftest import is_not_utc from data_gen import * from marks import allow_non_gpu, ignore_order from spark_session import is_before_spark_320 diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index d1f34826d65..5498a37b0d3 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -809,7 +809,6 @@ def test_read_case_col_name(spark_tmp_path, v1_enabled_list, col_name): pytest.param(True, marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/9517')), False ]) - @pytest.mark.parametrize('timezone', [ 'UTC', 'Etc/UTC', diff --git a/integration_tests/src/main/python/limit_test.py b/integration_tests/src/main/python/limit_test.py index 123b40785fe..5e116b00654 100644 --- a/integration_tests/src/main/python/limit_test.py +++ b/integration_tests/src/main/python/limit_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect -from conftest import is_not_utc from data_gen import * from spark_session import is_before_spark_340 from marks import allow_non_gpu, approximate_float diff --git a/integration_tests/src/main/python/row-based_udf_test.py b/integration_tests/src/main/python/row-based_udf_test.py index 118110ba1c8..e849a87b10e 100644 --- a/integration_tests/src/main/python/row-based_udf_test.py +++ b/integration_tests/src/main/python/row-based_udf_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_sql -from conftest import is_not_utc from data_gen import * from spark_session import with_spark_session, is_spark_350_or_later from conftest import skip_unless_precommit_tests diff --git a/integration_tests/src/main/python/row_conversion_test.py b/integration_tests/src/main/python/row_conversion_test.py index 2a559d28cba..92ea05d68be 100644 --- a/integration_tests/src/main/python/row_conversion_test.py +++ b/integration_tests/src/main/python/row_conversion_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect -from conftest import is_not_utc from data_gen import * from marks import allow_non_gpu, approximate_float, incompat from pyspark.sql.types import * diff --git a/integration_tests/src/main/python/sample_test.py b/integration_tests/src/main/python/sample_test.py index d1769ccef03..fc9d9fc4cbf 100644 --- a/integration_tests/src/main/python/sample_test.py +++ b/integration_tests/src/main/python/sample_test.py @@ -14,7 +14,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect -from conftest import is_not_utc from data_gen import * from pyspark.sql.types import * from spark_session import is_before_spark_330 diff --git a/integration_tests/src/main/python/struct_test.py b/integration_tests/src/main/python/struct_test.py index 18464089d9c..0e230a95408 100644 --- a/integration_tests/src/main/python/struct_test.py +++ b/integration_tests/src/main/python/struct_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_are_equal_sql -from conftest import is_not_utc from data_gen import * from pyspark.sql.types import * diff --git a/integration_tests/src/main/python/subquery_test.py b/integration_tests/src/main/python/subquery_test.py index c58c81caf1d..e6d641d4212 100644 --- a/integration_tests/src/main/python/subquery_test.py +++ b/integration_tests/src/main/python/subquery_test.py @@ -14,7 +14,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_sql -from conftest import is_not_utc from data_gen import * from marks import * diff --git a/integration_tests/src/main/python/time_window_test.py b/integration_tests/src/main/python/time_window_test.py index 318299742c1..ff367b506fb 100644 --- a/integration_tests/src/main/python/time_window_test.py +++ b/integration_tests/src/main/python/time_window_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect -from conftest import is_not_utc from data_gen import * from datetime import datetime from marks import ignore_order, allow_non_gpu diff --git a/integration_tests/src/main/python/udf_test.py b/integration_tests/src/main/python/udf_test.py index 1f530e5bb47..db8425f6387 100644 --- a/integration_tests/src/main/python/udf_test.py +++ b/integration_tests/src/main/python/udf_test.py @@ -14,7 +14,7 @@ import pytest -from conftest import is_at_least_precommit_run, is_not_utc +from conftest import is_at_least_precommit_run from spark_session import is_databricks_runtime, is_before_spark_330, is_before_spark_350, is_spark_340_or_later from pyspark.sql.pandas.utils import require_minimum_pyarrow_version, require_minimum_pandas_version diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index 681f7d3d6f1..3de2d3de859 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -15,7 +15,6 @@ import pytest from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_are_equal_sql, assert_gpu_fallback_collect, assert_gpu_sql_fallback_collect -from conftest import is_not_utc from data_gen import * from marks import * from pyspark.sql.types import * diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index d9496ff793b..e5cf75e9a13 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -1112,7 +1112,7 @@ abstract class BaseExprMeta[INPUT <: Expression]( // There are 4 levels of timezone check in GPU plan tag phase: // Level 1: Check whether an expression is related to timezone. This is achieved by // [[needTimeZoneCheck]] below. - // Level 2: Check on golden configuration 'spark.rapids.sql.nonUtc.enabled'. If + // Level 2: Check on golden configuration 'spark.rapids.sql.nonUTC.enabled'. If // yes, we pass to next level timezone check. If not, we only pass UTC case as before. // Level 3: Check related expression has been implemented with timezone. There is a // toggle flag [[isTimeZoneSupported]] for this. If false, fallback to UTC-only check as @@ -1123,7 +1123,8 @@ abstract class BaseExprMeta[INPUT <: Expression]( // Level 1 check if (!needTimeZoneCheck) return - // TODO: Level 2 check + // Level 2 check + if(!conf.nonUTCTimeZoneEnabled) return checkUTCTimezone(this) // Level 3 check if (!isTimeZoneSupported) return checkUTCTimezone(this) From 4d2935009141904aa4d4440e8d54d65d9b369826 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Fri, 1 Dec 2023 00:26:14 +0800 Subject: [PATCH 42/46] Fix UTC cases --- integration_tests/src/main/python/csv_test.py | 6 ++--- .../src/main/python/date_time_test.py | 3 +-- .../src/main/python/hash_aggregate_test.py | 9 +++---- .../src/main/python/json_test.py | 25 ++++++++++++------- 4 files changed, 23 insertions(+), 20 deletions(-) diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index 9b9c51f06fa..fe1dd259d17 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -331,10 +331,10 @@ def test_csv_fallback(spark_tmp_path, read_func, disable_conf, spark_tmp_table_f @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) @pytest.mark.parametrize('time_parser_policy', [ pytest.param('LEGACY', marks=pytest.mark.allow_non_gpu('BatchScanExec,FileSourceScanExec')), - 'CORRECTED', - 'EXCEPTION' + # Date is also time zone related for csv since rebase. + pytest.param('CORRECTED', marks=pytest.mark.allow_non_gpu(*non_utc_allow)), + pytest.param('EXCEPTION', marks=pytest.mark.allow_non_gpu(*non_utc_allow)) ]) -@allow_non_gpu(*non_utc_allow) # Date is also time zone related for csv since rebasing. def test_date_formats_round_trip(spark_tmp_path, date_format, v1_enabled_list, ansi_enabled, time_parser_policy): gen = StructGen([('a', DateGen())], nullable=False) data_path = spark_tmp_path + '/CSV_DATA' diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index d5063f246fa..4c08d7d3935 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -514,9 +514,8 @@ def test_date_format_maybe_incompat(data_gen, date_format): # where we had a failure due to GpuCast canonicalization with timezone. # In this case it was doing filter after project, the way I get that to happen is by adding in the # input_file_name(), otherwise filter happens before project. -@allow_non_gpu('CollectLimitExec,FileSourceScanExec,DeserializeToObjectExec') +@allow_non_gpu("CollectLimitExec", "FileSourceScanExec" ,"DeserializeToObjectExec", *non_utc_allow) @ignore_order() -@allow_non_gpu(*non_utc_allow) def test_date_format_mmyyyy_cast_canonicalization(spark_tmp_path): data_path = spark_tmp_path + '/CSV_DATA' gen = StringGen(pattern='[0][0-9][1][8-9][1-9][1-9]', nullable=False) diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index c42b5dffaf7..c8d304c4c93 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -627,11 +627,10 @@ def test_min_max_group_by(data_gen): # support sorting certain nested/arbitrary types on the GPU # See https://github.com/NVIDIA/spark-rapids/issues/3715 # and https://github.com/rapidsai/cudf/issues/11222 -@allow_non_gpu("ProjectExec", "SortArray") +@allow_non_gpu("ProjectExec", "SortArray", *non_utc_allow) @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _gen_data_for_collect_list_op, ids=idfn) @pytest.mark.parametrize('use_obj_hash_agg', [True, False], ids=idfn) -@allow_non_gpu(*non_utc_allow) def test_hash_groupby_collect_list(data_gen, use_obj_hash_agg): def doit(spark): df = gen_df(spark, data_gen, length=100)\ @@ -685,9 +684,8 @@ def test_hash_groupby_collect_set_on_nested_type(data_gen): # See https://github.com/NVIDIA/spark-rapids/issues/3715 # and https://github.com/rapidsai/cudf/issues/11222 @ignore_order(local=True) -@allow_non_gpu("ProjectExec", "SortArray") +@allow_non_gpu("ProjectExec", "SortArray", *non_utc_allow) @pytest.mark.parametrize('data_gen', _gen_data_for_collect_set_op_nested, ids=idfn) -@allow_non_gpu(*non_utc_allow) def test_hash_groupby_collect_set_on_nested_array_type(data_gen): conf = copy_and_update(_float_conf, { "spark.rapids.sql.castFloatToString.enabled": "true", @@ -729,9 +727,8 @@ def test_hash_reduction_collect_set_on_nested_type(data_gen): # See https://github.com/NVIDIA/spark-rapids/issues/3715 # and https://github.com/rapidsai/cudf/issues/11222 @ignore_order(local=True) -@allow_non_gpu("ProjectExec", "SortArray") +@allow_non_gpu("ProjectExec", "SortArray", *non_utc_allow) @pytest.mark.parametrize('data_gen', _gen_data_for_collect_set_op_nested, ids=idfn) -@allow_non_gpu(*non_utc_allow) def test_hash_reduction_collect_set_on_nested_array_type(data_gen): conf = copy_and_update(_float_conf, { "spark.rapids.sql.castFloatToString.enabled": "true", diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 506ce6a06f1..23ec43ae43d 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -394,6 +394,11 @@ def test_json_read_invalid_dates(std_input_path, filename, schema, read_func, an else: assert_gpu_and_cpu_are_equal_collect(f, conf=updated_conf) +# allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' +non_utc_file_source_scan_allow = ['FileSourceScanExec'] if is_not_utc() else [] + +non_utc_project_allow = ['ProjectExec'] if is_not_utc() else [] + @approximate_float @pytest.mark.parametrize('filename', [ 'timestamps.json', @@ -403,10 +408,10 @@ def test_json_read_invalid_dates(std_input_path, filename, schema, read_func, an @pytest.mark.parametrize('ansi_enabled', ["true", "false"]) @pytest.mark.parametrize('time_parser_policy', [ pytest.param('LEGACY', marks=pytest.mark.allow_non_gpu('FileSourceScanExec')), - 'CORRECTED', - 'EXCEPTION' + # For non UTC cases, corrected and exception will have CPU fallback in lack of timezone support. + pytest.param('CORRECTED', marks=pytest.mark.allow_non_gpu(*non_utc_file_source_scan_allow)), + pytest.param('EXCEPTION', marks=pytest.mark.allow_non_gpu(*non_utc_file_source_scan_allow)) ]) -@allow_non_gpu(*non_utc_allow) def test_json_read_valid_timestamps(std_input_path, filename, schema, read_func, ansi_enabled, time_parser_policy, \ spark_tmp_table_factory): updated_conf = copy_and_update(_enable_all_types_conf, @@ -560,8 +565,8 @@ def test_from_json_struct_decimal(): "(true|false)" ]) @pytest.mark.parametrize('date_format', [ - "", - "yyyy-MM-dd", + pytest.param("", marks=pytest.mark.allow_non_gpu(*non_utc_project_allow)), + pytest.param("yyyy-MM-dd", marks=pytest.mark.allow_non_gpu(*non_utc_project_allow)), # https://github.com/NVIDIA/spark-rapids/issues/9667 pytest.param("dd/MM/yyyy", marks=pytest.mark.allow_non_gpu('ProjectExec')), ]) @@ -569,7 +574,6 @@ def test_from_json_struct_decimal(): pytest.param("LEGACY", marks=pytest.mark.allow_non_gpu('ProjectExec')), "CORRECTED" ]) -@allow_non_gpu(*non_utc_allow) def test_from_json_struct_date(date_gen, date_format, time_parser_policy): json_string_gen = StringGen(r'{ "a": ' + date_gen + ' }') \ .with_special_case('{ "a": null }') \ @@ -617,6 +621,9 @@ def test_from_json_struct_date_fallback_non_default_format(date_gen, date_format conf={"spark.rapids.sql.expression.JsonToStructs": True, 'spark.sql.legacy.timeParserPolicy': 'CORRECTED'}) +# allow non gpu when time zone is non-UTC because of https://github.com/NVIDIA/spark-rapids/issues/9653' +non_utc_project_allow = ['ProjectExec'] if is_not_utc() else [] + @pytest.mark.parametrize('timestamp_gen', [ # "yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]" "\"[ \t\xA0\u1680\u180e\u2000-\u200a\u202f\u205f\u3000]?[1-8]{1}[0-9]{3}-[0-3]{1,2}-[0-3]{1,2}T[0-9]{1,2}:[0-9]{1,2}:[0-9]{1,2}(\\.[0-9]{1,6})?Z?[ \t\xA0\u1680\u180e\u2000-\u200a\u202f\u205f\u3000]}?\"", @@ -641,8 +648,9 @@ def test_from_json_struct_date_fallback_non_default_format(date_gen, date_format "(true|false)" ]) @pytest.mark.parametrize('timestamp_format', [ - "", - "yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]", + # Even valid timestamp format, CPU fallback happens still since non UTC is not supported for json. + pytest.param("", marks=pytest.mark.allow_non_gpu(*non_utc_project_allow)), + pytest.param("yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]", marks=pytest.mark.allow_non_gpu(*non_utc_project_allow)), # https://github.com/NVIDIA/spark-rapids/issues/9723 pytest.param("yyyy-MM-dd'T'HH:mm:ss.SSSXXX", marks=pytest.mark.allow_non_gpu('ProjectExec')), pytest.param("dd/MM/yyyy'T'HH:mm:ss[.SSS][XXX]", marks=pytest.mark.allow_non_gpu('ProjectExec')), @@ -652,7 +660,6 @@ def test_from_json_struct_date_fallback_non_default_format(date_gen, date_format "CORRECTED" ]) @pytest.mark.parametrize('ansi_enabled', [ True, False ]) -@allow_non_gpu(*non_utc_allow) def test_from_json_struct_timestamp(timestamp_gen, timestamp_format, time_parser_policy, ansi_enabled): json_string_gen = StringGen(r'{ "a": ' + timestamp_gen + ' }') \ .with_special_case('{ "a": null }') \ From 950ac3c5beacd96b9fb268a6961e0f736fb4e25b Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Thu, 30 Nov 2023 17:40:54 +0000 Subject: [PATCH 43/46] Address comments --- integration_tests/src/main/python/csv_test.py | 32 ++++++++++++------- .../src/main/python/json_test.py | 17 ++++++---- 2 files changed, 31 insertions(+), 18 deletions(-) diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index fe1dd259d17..196c997c1e3 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -573,7 +573,6 @@ def test_csv_read_count(spark_tmp_path): @pytest.mark.parametrize("timestamp_type", [ pytest.param('TIMESTAMP_LTZ', marks=pytest.mark.xfail(is_spark_350_or_later(), reason="https://github.com/NVIDIA/spark-rapids/issues/9325")), "TIMESTAMP_NTZ"]) -@pytest.mark.xfail(is_not_utc(), reason='Timezone is not supported for csv format as https://github.com/NVIDIA/spark-rapids/issues/9653.') def test_csv_infer_schema_timestamp_ntz_v1(spark_tmp_path, date_format, ts_part, timestamp_type): csv_infer_schema_timestamp_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, 'csv', 'FileSourceScanExec') @@ -618,15 +617,19 @@ def do_read(spark): cpu_fallback_class_name = cpu_scan_class, conf = conf) else: - assert_cpu_and_gpu_are_equal_collect_with_capture( - lambda spark: do_read(spark), - exist_classes = 'Gpu' + cpu_scan_class, - non_exist_classes = cpu_scan_class, - conf = conf) + if is_not_utc(): # non UTC is not support for csv, skip capture check + assert_gpu_and_cpu_are_equal_collect(lambda spark: do_read(spark), conf = conf) + else: + assert_cpu_and_gpu_are_equal_collect_with_capture( + lambda spark: do_read(spark), + exist_classes = 'Gpu' + cpu_scan_class, + non_exist_classes = cpu_scan_class, + conf = conf) + + @allow_non_gpu('FileSourceScanExec', 'CollectLimitExec', 'DeserializeToObjectExec') @pytest.mark.skipif(is_before_spark_340(), reason='`preferDate` is only supported in Spark 340+') -@pytest.mark.xfail(is_not_utc(), reason='Timezone is not supported for csv format as https://github.com/NVIDIA/spark-rapids/issues/9653.') def test_csv_prefer_date_with_infer_schema(spark_tmp_path): # start date ""0001-01-02" required due to: https://github.com/NVIDIA/spark-rapids/issues/5606 data_gens = [byte_gen, short_gen, int_gen, long_gen, boolean_gen, timestamp_gen, DateGen(start=date(1, 1, 2))] @@ -635,14 +638,21 @@ def test_csv_prefer_date_with_infer_schema(spark_tmp_path): with_cpu_session(lambda spark: gen_df(spark, gen_list).write.csv(data_path)) + if is_not_utc(): # non UTC is not support for csv + exist_clazz = 'FileSourceScanExec' + non_exist_clazz = 'GpuFileSourceScanExec' + else: + exist_clazz = 'GpuFileSourceScanExec' + non_exist_clazz = 'FileSourceScanExec' + assert_cpu_and_gpu_are_equal_collect_with_capture( lambda spark: spark.read.option("inferSchema", "true").csv(data_path), - exist_classes = 'GpuFileSourceScanExec', - non_exist_classes = 'FileSourceScanExec') + exist_classes = exist_clazz, + non_exist_classes = non_exist_clazz) assert_cpu_and_gpu_are_equal_collect_with_capture( lambda spark: spark.read.option("inferSchema", "true").option("preferDate", "false").csv(data_path), - exist_classes = 'GpuFileSourceScanExec', - non_exist_classes = 'FileSourceScanExec') + exist_classes = exist_clazz, + non_exist_classes = non_exist_clazz) @allow_non_gpu('FileSourceScanExec') @pytest.mark.skipif(is_before_spark_340(), reason='enableDateTimeParsingFallback is supported from Spark3.4.0') diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 23ec43ae43d..d95b67dd49c 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -208,7 +208,6 @@ def test_json_ts_formats_round_trip(spark_tmp_path, date_format, ts_part, v1_ena @allow_non_gpu('FileSourceScanExec', 'ProjectExec') @pytest.mark.skipif(is_before_spark_341(), reason='`TIMESTAMP_NTZ` is only supported in PySpark 341+.') -@pytest.mark.xfail(is_not_utc(), reason='Timezone is not supported for json format as https://github.com/NVIDIA/spark-rapids/issues/9653.') @pytest.mark.parametrize('ts_part', json_supported_ts_parts) @pytest.mark.parametrize('date_format', json_supported_date_formats) @pytest.mark.parametrize("timestamp_type", ["TIMESTAMP_LTZ", "TIMESTAMP_NTZ"]) @@ -217,7 +216,6 @@ def test_json_ts_formats_round_trip_ntz_v1(spark_tmp_path, date_format, ts_part, @allow_non_gpu('BatchScanExec', 'ProjectExec') @pytest.mark.skipif(is_before_spark_341(), reason='`TIMESTAMP_NTZ` is only supported in PySpark 341+.') -@pytest.mark.xfail(is_not_utc(), reason='Timezone is not supported for json format as https://github.com/NVIDIA/spark-rapids/issues/9653.') @pytest.mark.parametrize('ts_part', json_supported_ts_parts) @pytest.mark.parametrize('date_format', json_supported_date_formats) @pytest.mark.parametrize("timestamp_type", ["TIMESTAMP_LTZ", "TIMESTAMP_NTZ"]) @@ -248,11 +246,16 @@ def do_read(spark): if timestamp_type == "TIMESTAMP_LTZ": - assert_cpu_and_gpu_are_equal_collect_with_capture( - lambda spark : do_read(spark), - exist_classes = 'Gpu' + cpu_scan_class, - non_exist_classes = cpu_scan_class, - conf=updated_conf) + if is_not_utc(): # non UTC is not support for csv, skip capture check + assert_gpu_and_cpu_are_equal_collect(lambda spark: do_read(spark), conf = updated_conf) + else: + assert_cpu_and_gpu_are_equal_collect_with_capture( + lambda spark : do_read(spark), + exist_classes = 'Gpu' + cpu_scan_class, + non_exist_classes = cpu_scan_class, + conf=updated_conf) + + else: # we fall back to CPU due to "unsupported data types in output: TimestampNTZType" assert_gpu_fallback_collect( From eb0c85e19e9e15e6fb92bf5180ae396ee86d8326 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Fri, 1 Dec 2023 08:48:50 +0800 Subject: [PATCH 44/46] Address comments --- integration_tests/src/main/python/csv_test.py | 4 +++- integration_tests/src/main/python/json_test.py | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index 196c997c1e3..3fb83a665e3 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -617,7 +617,9 @@ def do_read(spark): cpu_fallback_class_name = cpu_scan_class, conf = conf) else: - if is_not_utc(): # non UTC is not support for csv, skip capture check + if is_not_utc(): + # non UTC is not support for csv, skip capture check + # tracked in https://github.com/NVIDIA/spark-rapids/issues/9913 assert_gpu_and_cpu_are_equal_collect(lambda spark: do_read(spark), conf = conf) else: assert_cpu_and_gpu_are_equal_collect_with_capture( diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index d95b67dd49c..d8aceb3e705 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -246,7 +246,9 @@ def do_read(spark): if timestamp_type == "TIMESTAMP_LTZ": - if is_not_utc(): # non UTC is not support for csv, skip capture check + if is_not_utc(): + # non UTC is not support for json, skip capture check + # Tracked in https://github.com/NVIDIA/spark-rapids/issues/9912 assert_gpu_and_cpu_are_equal_collect(lambda spark: do_read(spark), conf = updated_conf) else: assert_cpu_and_gpu_are_equal_collect_with_capture( From 0267c81521739d5c0d977534ac15333496f60d6c Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Fri, 1 Dec 2023 13:55:54 +0800 Subject: [PATCH 45/46] Fix a merge issue --- integration_tests/src/main/python/udf_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/udf_test.py b/integration_tests/src/main/python/udf_test.py index 3cedb6400db..5ce99318b03 100644 --- a/integration_tests/src/main/python/udf_test.py +++ b/integration_tests/src/main/python/udf_test.py @@ -15,7 +15,7 @@ import pytest from conftest import is_at_least_precommit_run, is_not_utc -from spark_session import is_databricks_runtime, is_before_spark_330, is_before_spark_350, is_spark_340_or_later +from spark_session import is_databricks_runtime, is_before_spark_330, is_before_spark_350, is_spark_341 from pyspark.sql.pandas.utils import require_minimum_pyarrow_version, require_minimum_pandas_version From 216daf3d29d358337bee8c89c23c06e75cef1488 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Fri, 1 Dec 2023 13:59:47 +0800 Subject: [PATCH 46/46] Minor fix --- integration_tests/src/main/python/udf_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/udf_test.py b/integration_tests/src/main/python/udf_test.py index 5ce99318b03..0604b74585d 100644 --- a/integration_tests/src/main/python/udf_test.py +++ b/integration_tests/src/main/python/udf_test.py @@ -14,7 +14,7 @@ import pytest -from conftest import is_at_least_precommit_run, is_not_utc +from conftest import is_at_least_precommit_run from spark_session import is_databricks_runtime, is_before_spark_330, is_before_spark_350, is_spark_341 from pyspark.sql.pandas.utils import require_minimum_pyarrow_version, require_minimum_pandas_version