From 79a72d5b80978fff4e0228ba1481fac65722e9b3 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 19 Apr 2024 16:12:04 -0700 Subject: [PATCH 01/15] AnalysisException child class Signed-off-by: Raza Jafri --- .../sql/hive/rapids/RapidsHiveErrors.scala | 4 +-- .../spark/sql/rapids/GpuDataSourceBase.scala | 26 +++++++++---------- ...GpuInsertIntoHadoopFsRelationCommand.scala | 4 +-- .../expressions/GpuRandomExpressions.scala | 4 +-- .../sql/rapids/execution/TrampolineUtil.scala | 2 +- .../rapids/shims/GpuInsertIntoHiveTable.scala | 2 +- .../sql/rapids/GpuFileFormatWriter.scala | 2 +- ...eDataSourceTableAsSelectCommandShims.scala | 3 ++- 8 files changed, 24 insertions(+), 23 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala index 259a04ec318..cfeda70ec3b 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -54,7 +54,7 @@ object RapidsHiveErrors { def cannotResolveAttributeError(name: String, outputStr: String): Throwable = { new AnalysisException( - s"Unable to resolve $name given [$outputStr]") + s"Unable to resolve $name given [$outputStr]") {} } def writePartitionExceedConfigSizeWhenDynamicPartitionError( diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala index 0ec720733e8..4439b68daf0 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -145,7 +145,7 @@ abstract class GpuDataSourceBase( inferredOpt }.getOrElse { throw new AnalysisException(s"Failed to resolve the schema for $format for " + - s"the partition column: $partitionColumn. It must be specified manually.") + s"the partition column: $partitionColumn. It must be specified manually.") {} } } StructType(partitionFields) @@ -163,7 +163,7 @@ abstract class GpuDataSourceBase( SparkShimImpl.filesFromFileIndex(tempFileIndex)) }.getOrElse { throw new AnalysisException( - s"Unable to infer schema for $format. It must be specified manually.") + s"Unable to infer schema for $format. It must be specified manually.") {} } // We just print a waring message if the data schema and partition schema have the duplicate @@ -201,7 +201,7 @@ abstract class GpuDataSourceBase( case (dataSource: RelationProvider, None) => dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) case (_: SchemaRelationProvider, None) => - throw new AnalysisException(s"A schema needs to be specified when using $className.") + throw new AnalysisException(s"A schema needs to be specified when using $className.") {} case (dataSource: RelationProvider, Some(schema)) => val baseRelation = dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) @@ -211,7 +211,7 @@ abstract class GpuDataSourceBase( s"user-specified: ${schema.toDDL}, actual: ${baseRelation.schema.toDDL}. If " + "you're using DataFrameReader.schema API or creating a table, please do not " + "specify the schema. Or if you're scanning an existed table, please drop " + - "it and re-create it.") + "it and re-create it.") {} } baseRelation @@ -235,7 +235,7 @@ abstract class GpuDataSourceBase( }.getOrElse { throw new AnalysisException( s"Unable to infer schema for $format at ${fileCatalog.allFiles().mkString(",")}. " + - "It must be specified manually") + "It must be specified manually") {} } HadoopFsRelation( @@ -277,7 +277,7 @@ abstract class GpuDataSourceBase( case _ => throw new AnalysisException( - s"$className is not a valid Spark SQL Data Source.") + s"$className is not a valid Spark SQL Data Source.") {} } relation match { @@ -414,19 +414,19 @@ object GpuDataSourceBase extends Logging { throw new AnalysisException( "Hive built-in ORC data source must be used with Hive support enabled. " + "Please use the native ORC data source by setting 'spark.sql.orc.impl' to " + - "'native'") + "'native'") {} } else if (provider1.toLowerCase(Locale.ROOT) == "avro" || provider1 == "com.databricks.spark.avro" || provider1 == "org.apache.spark.sql.avro") { throw new AnalysisException( s"Failed to find data source: $provider1. Avro is built-in but external data " + "source module since Spark 2.4. Please deploy the application as per " + - "the deployment section of \"Apache Avro Data Source Guide\".") + "the deployment section of \"Apache Avro Data Source Guide\".") {} } else if (provider1.toLowerCase(Locale.ROOT) == "kafka") { throw new AnalysisException( s"Failed to find data source: $provider1. Please deploy the application as " + "per the deployment section of " + - "\"Structured Streaming + Kafka Integration Guide\".") + "\"Structured Streaming + Kafka Integration Guide\".") {} } else { throw new ClassNotFoundException( s"Failed to find data source: $provider1. Please find packages at " + @@ -460,7 +460,7 @@ object GpuDataSourceBase extends Logging { internalSources.head.getClass } else { throw new AnalysisException(s"Multiple sources found for $provider1 " + - s"(${sourceNames.mkString(", ")}), please specify the fully qualified class name.") + s"(${sourceNames.mkString(", ")}), please specify the fully qualified class name.") {} } } } catch { @@ -513,7 +513,7 @@ object GpuDataSourceBase extends Logging { } if (checkEmptyGlobPath && globResult.isEmpty) { - throw new AnalysisException(s"Path does not exist: $globPath") + throw new AnalysisException(s"Path does not exist: $globPath") {} } globResult @@ -527,7 +527,7 @@ object GpuDataSourceBase extends Logging { ThreadUtils.parmap(nonGlobPaths, "checkPathsExist", numThreads) { path => val fs = path.getFileSystem(hadoopConf) if (!fs.exists(path)) { - throw new AnalysisException(s"Path does not exist: $path") + throw new AnalysisException(s"Path does not exist: $path") {} } } } catch { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala index 2b7974fd1a6..4627538b3f1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2022, NVIDIA CORPORATION. + * Copyright (c) 2019-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -121,7 +121,7 @@ case class GpuInsertIntoHadoopFsRelationCommand( val pathExists = fs.exists(qualifiedOutputPath) (mode, pathExists) match { case (SaveMode.ErrorIfExists, true) => - throw new AnalysisException(s"path $qualifiedOutputPath already exists.") + throw new AnalysisException(s"path $qualifiedOutputPath already exists.") {} case (SaveMode.Overwrite, true) => if (ifPartitionNotExists && matchingPartitions.nonEmpty) { false diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala index 6675f678f6d..59fd03f80c9 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -53,7 +53,7 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres case GpuLiteral(s, IntegerType) => s.asInstanceOf[Int] case GpuLiteral(s, LongType) => s.asInstanceOf[Long] case _ => throw new AnalysisException( - s"Input argument to $prettyName must be an integer, long or null literal.") + s"Input argument to $prettyName must be an integer, long or null literal.") {} } @transient protected var previousPartition: Int = 0 diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala index eb16edc0b51..7be64ee2e0d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala @@ -157,7 +157,7 @@ object TrampolineUtil { } /** Throw a Spark analysis exception */ - def throwAnalysisException(msg: String) = throw new AnalysisException(msg) + def throwAnalysisException(msg: String) = throw new AnalysisException(msg) {} /** Set the task context for the current thread */ def setTaskContext(tc: TaskContext): Unit = TaskContext.setTaskContext(tc) 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 9105ab50e1e..095301c238f 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 @@ -181,7 +181,7 @@ case class GpuInsertIntoHiveTable( // Report error if any static partition appears after a dynamic partition val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) if (isDynamic.init.zip(isDynamic.tail).contains((true, false))) { - throw new AnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) + throw new AnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) {} } } diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index 78daa0bf6f1..0e1e88a2b8f 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -62,7 +62,7 @@ object GpuFileFormatWriter extends Logging { schema.foreach { field => if (!format.supportDataType(field.dataType)) { throw new AnalysisException( - s"$format data source does not support ${field.dataType.catalogString} data type.") + s"$format data source does not support ${field.dataType.catalogString} data type.") {} } } } diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala index 9e36cf41fad..7a514d35b76 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala @@ -64,7 +64,8 @@ case class GpuCreateDataSourceTableAsSelectCommand( s"Expect the table $tableName has been dropped when the save mode is Overwrite") if (mode == SaveMode.ErrorIfExists) { - throw new AnalysisException(s"Table $tableName already exists. You need to drop it first.") + throw new AnalysisException(s"Table $tableName already exists. " + + "You need to drop it first.") {} } if (mode == SaveMode.Ignore) { // Since the table already exists and the save mode is Ignore, we will just return. From 72caee01a7a1700db897bb25c09c0f21f7098e2e Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 17 May 2024 18:20:54 -0700 Subject: [PATCH 02/15] Use errorClass for reporting AnalysisException --- .../rapids/GpuDataWritingCommandExec.scala | 8 +- .../spark/rapids/GpuRunnableCommandExec.scala | 8 +- .../sql/hive/rapids/RapidsHiveErrors.scala | 7 +- .../spark/sql/rapids/GpuDataSourceBase.scala | 58 ++++----- ...GpuInsertIntoHadoopFsRelationCommand.scala | 6 +- .../expressions/GpuRandomExpressions.scala | 6 +- .../sql/rapids/execution/TrampolineUtil.scala | 5 +- .../rapids/shims/ParquetSchemaClipShims.scala | 15 +-- .../rapids/shims/GpuInsertIntoHiveTable.scala | 5 +- .../rapids/shims/AnalysisExceptionShim.scala | 114 ++++++++++++++++++ .../rapids/shims/ParquetSchemaClipShims.scala | 14 ++- .../rapids/shims/ParquetSchemaClipShims.scala | 14 ++- .../rapids/shims/GpuInsertIntoHiveTable.scala | 5 +- .../sql/rapids/GpuFileFormatWriter.scala | 7 +- ...eDataSourceTableAsSelectCommandShims.scala | 4 +- .../rapids/shims/AnalysisExceptionShim.scala | 36 ++++++ 16 files changed, 228 insertions(+), 84 deletions(-) create mode 100644 sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala create mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala index 5a54d0b2f66..3452b976a94 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.command.DataWritingCommand import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.GpuWriteJobStatsTracker -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration @@ -84,10 +84,8 @@ object GpuDataWritingCommand { if (fs.exists(filePath) && fs.getFileStatus(filePath).isDirectory && fs.listStatus(filePath).length != 0) { - TrampolineUtil.throwAnalysisException( - s"CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory " + - s"${tablePath} . To allow overwriting the existing non-empty directory, " + - s"set '$allowNonEmptyLocationInCTASKey' to true.") + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1241", + Map("tablePath" -> s"$tablePath", "config" -> s"$allowNonEmptyLocationInCTASKey")) } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala index e3869960fc4..c2d392fb473 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.GpuWriteJobStatsTracker -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration @@ -82,10 +82,8 @@ object GpuRunnableCommand { if (fs.exists(filePath) && fs.getFileStatus(filePath).isDirectory && fs.listStatus(filePath).length != 0) { - TrampolineUtil.throwAnalysisException( - s"CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory " + - s"${tablePath} . To allow overwriting the existing non-empty directory, " + - s"set '$allowNonEmptyLocationInCTASKey' to true.") + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1241", + Map("tablePath" -> s"$tablePath", "config" -> s"$allowNonEmptyLocationInCTASKey")) } } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala index cfeda70ec3b..9ac7aafd98a 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.hive.rapids import org.apache.hadoop.fs.Path import org.apache.spark.SparkException -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim import org.apache.spark.sql.types.{DataType, DoubleType, FloatType, StringType} object RapidsHiveErrors { @@ -53,8 +53,9 @@ object RapidsHiveErrors { } def cannotResolveAttributeError(name: String, outputStr: String): Throwable = { - new AnalysisException( - s"Unable to resolve $name given [$outputStr]") {} + AnalysisExceptionShim.throwException( + errorClass = "_LEGACY_ERROR_TEMP_1137", + Map("name" -> s"$name", "outputStr" -> s"$outputStr")) } def writePartitionExceedConfigSizeWhenDynamicPartitionError( diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala index 4439b68daf0..9ebd16fa008 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.shims.SchemaUtilsShims +import org.apache.spark.sql.rapids.shims.{AnalysisExceptionShim, SchemaUtilsShims} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.{HadoopFSUtils, ThreadUtils, Utils} @@ -144,8 +144,8 @@ abstract class GpuDataSourceBase( } inferredOpt }.getOrElse { - throw new AnalysisException(s"Failed to resolve the schema for $format for " + - s"the partition column: $partitionColumn. It must be specified manually.") {} + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1128", + Map("format" -> s"$format", "partitionColumn" -> s"$partitionColumn")) } } StructType(partitionFields) @@ -162,8 +162,8 @@ abstract class GpuDataSourceBase( caseInsensitiveOptions - "path", SparkShimImpl.filesFromFileIndex(tempFileIndex)) }.getOrElse { - throw new AnalysisException( - s"Unable to infer schema for $format. It must be specified manually.") {} + AnalysisExceptionShim. + throwException("UNABLE_TO_INFER_SCHEMA", Map("format" -> s"$format")) } // We just print a waring message if the data schema and partition schema have the duplicate @@ -201,17 +201,14 @@ abstract class GpuDataSourceBase( case (dataSource: RelationProvider, None) => dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) case (_: SchemaRelationProvider, None) => - throw new AnalysisException(s"A schema needs to be specified when using $className.") {} + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1132", + Map("className" -> s"$className")) case (dataSource: RelationProvider, Some(schema)) => val baseRelation = dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) if (!DataType.equalsIgnoreCompatibleNullability(baseRelation.schema, schema)) { - throw new AnalysisException( - "The user-specified schema doesn't match the actual schema: " + - s"user-specified: ${schema.toDDL}, actual: ${baseRelation.schema.toDDL}. If " + - "you're using DataFrameReader.schema API or creating a table, please do not " + - "specify the schema. Or if you're scanning an existed table, please drop " + - "it and re-create it.") {} + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1133", + Map("schema" -> s"${schema.toDDL}", "actualSchema" -> s"${baseRelation.schema.toDDL}")) } baseRelation @@ -233,9 +230,9 @@ abstract class GpuDataSourceBase( caseInsensitiveOptions - "path", SparkShimImpl.filesFromFileIndex(fileCatalog)) }.getOrElse { - throw new AnalysisException( - s"Unable to infer schema for $format at ${fileCatalog.allFiles().mkString(",")}. " + - "It must be specified manually") {} + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1134", + Map("format" -> s"$format", + "fileCatalog" -> s"${fileCatalog.allFiles().mkString(",")}")) } HadoopFsRelation( @@ -276,8 +273,8 @@ abstract class GpuDataSourceBase( caseInsensitiveOptions)(sparkSession) case _ => - throw new AnalysisException( - s"$className is not a valid Spark SQL Data Source.") {} + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1135", + Map("className" -> s"$className")) } relation match { @@ -411,22 +408,16 @@ object GpuDataSourceBase extends Logging { dataSource case Failure(error) => if (provider1.startsWith("org.apache.spark.sql.hive.orc")) { - throw new AnalysisException( - "Hive built-in ORC data source must be used with Hive support enabled. " + - "Please use the native ORC data source by setting 'spark.sql.orc.impl' to " + - "'native'") {} + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1138", + Map.empty[String, String]) } else if (provider1.toLowerCase(Locale.ROOT) == "avro" || provider1 == "com.databricks.spark.avro" || provider1 == "org.apache.spark.sql.avro") { - throw new AnalysisException( - s"Failed to find data source: $provider1. Avro is built-in but external data " + - "source module since Spark 2.4. Please deploy the application as per " + - "the deployment section of \"Apache Avro Data Source Guide\".") {} + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1139", + Map("provider" -> s"$provider1")) } else if (provider1.toLowerCase(Locale.ROOT) == "kafka") { - throw new AnalysisException( - s"Failed to find data source: $provider1. Please deploy the application as " + - "per the deployment section of " + - "\"Structured Streaming + Kafka Integration Guide\".") {} + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1140", + Map("provider" -> s"$provider1")) } else { throw new ClassNotFoundException( s"Failed to find data source: $provider1. Please find packages at " + @@ -459,8 +450,9 @@ object GpuDataSourceBase extends Logging { s"defaulting to the internal datasource (${internalSources.head.getClass.getName}).") internalSources.head.getClass } else { - throw new AnalysisException(s"Multiple sources found for $provider1 " + - s"(${sourceNames.mkString(", ")}), please specify the fully qualified class name.") {} + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1141", + Map("provider" -> s"$provider1", + "sourceNames" -> s"${sourceNames.mkString(", ")}")) } } } catch { @@ -513,7 +505,7 @@ object GpuDataSourceBase extends Logging { } if (checkEmptyGlobPath && globResult.isEmpty) { - throw new AnalysisException(s"Path does not exist: $globPath") {} + AnalysisExceptionShim.throwException("PATH_NOT_FOUND", Map("path" -> s"$globPath")) } globResult @@ -527,7 +519,7 @@ object GpuDataSourceBase extends Logging { ThreadUtils.parmap(nonGlobPaths, "checkPathsExist", numThreads) { path => val fs = path.getFileSystem(hadoopConf) if (!fs.exists(path)) { - throw new AnalysisException(s"Path does not exist: $path") {} + AnalysisExceptionShim.throwException("PATH_NOT_FOUND", Map("path" -> s"$path")) } } } catch { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala index 4627538b3f1..9c55784e462 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala @@ -22,7 +22,7 @@ import com.nvidia.spark.rapids.{ColumnarFileFormat, GpuDataWritingCommand} import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.internal.io.FileCommitProtocol -import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} +import org.apache.spark.sql.{SaveMode, SparkSession} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTablePartition} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.getPartitionPathString @@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.{AlterTableAddPartitionCommand, AlterTableDropPartitionCommand, CommandUtils} import org.apache.spark.sql.execution.datasources.{FileFormatWriter, FileIndex, PartitioningUtils} import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode -import org.apache.spark.sql.rapids.shims.SchemaUtilsShims +import org.apache.spark.sql.rapids.shims.{AnalysisExceptionShim, SchemaUtilsShims} import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuInsertIntoHadoopFsRelationCommand( @@ -121,7 +121,7 @@ case class GpuInsertIntoHadoopFsRelationCommand( val pathExists = fs.exists(qualifiedOutputPath) (mode, pathExists) match { case (SaveMode.ErrorIfExists, true) => - throw new AnalysisException(s"path $qualifiedOutputPath already exists.") {} + AnalysisExceptionShim.throwException(s"path $qualifiedOutputPath already exists.") case (SaveMode.Overwrite, true) => if (ifPartitionNotExists && matchingPartitions.nonEmpty) { false diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala index 59fd03f80c9..e60bd0692df 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala @@ -23,8 +23,8 @@ import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.shims.ShimUnaryExpression import org.apache.spark.TaskContext -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionWithRandomSeed} +import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils @@ -52,8 +52,8 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres @transient protected lazy val seed: Long = child match { case GpuLiteral(s, IntegerType) => s.asInstanceOf[Int] case GpuLiteral(s, LongType) => s.asInstanceOf[Long] - case _ => throw new AnalysisException( - s"Input argument to $prettyName must be an integer, long or null literal.") {} + case _ => AnalysisExceptionShim.throwException( + s"Input argument to $prettyName must be an integer, long or null literal.") } @transient protected var previousPartition: Int = 0 diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala index 7be64ee2e0d..cbd00177d7c 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala @@ -31,7 +31,7 @@ import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.scheduler.SparkListenerEvent import org.apache.spark.security.CryptoStreamUtils import org.apache.spark.serializer.{JavaSerializer, SerializerManager} -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode import org.apache.spark.sql.execution.SparkPlan @@ -156,9 +156,6 @@ object TrampolineUtil { TaskContext.get.taskMemoryManager() } - /** Throw a Spark analysis exception */ - def throwAnalysisException(msg: String) = throw new AnalysisException(msg) {} - /** Set the task context for the current thread */ def setTaskContext(tc: TaskContext): Unit = TaskContext.setTaskContext(tc) diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index fd48b8b6375..6195e70eb23 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -26,7 +26,7 @@ import org.apache.parquet.schema.OriginalType._ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim import org.apache.spark.sql.types._ object ParquetSchemaClipShims { @@ -64,13 +64,15 @@ object ParquetSchemaClipShims { if (originalType == null) s"$typeName" else s"$typeName ($originalType)" def typeNotSupported() = - TrampolineUtil.throwAnalysisException(s"Parquet type not supported: $typeString") + AnalysisExceptionShim.throwException(s"Parquet type not supported: $typeString") def typeNotImplemented() = - TrampolineUtil.throwAnalysisException(s"Parquet type not yet supported: $typeString") + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1172", + Map("parquetType" -> s"$typeString")) def illegalType() = - TrampolineUtil.throwAnalysisException(s"Illegal Parquet type: $typeString") + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1173", + Map("parquetType" -> s"$typeString")) // When maxPrecision = -1, we skip precision range check, and always respect the precision // specified in field.getDecimalMetadata. This is useful when interpreting decimal types stored @@ -80,8 +82,7 @@ object ParquetSchemaClipShims { val scale = field.getDecimalMetadata.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - TrampolineUtil.throwAnalysisException( - s"Invalid decimal precision: $typeName " + + AnalysisExceptionShim.throwException(s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -121,7 +122,7 @@ object ParquetSchemaClipShims { case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - TrampolineUtil.throwAnalysisException( + AnalysisExceptionShim.throwException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } 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 92fb72801c8..ca8a83c5930 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 @@ -45,7 +45,7 @@ import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.spark.SparkException -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, ExternalCatalog, ExternalCatalogUtils, ExternalCatalogWithListener} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -58,6 +58,7 @@ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.client.hive._ import org.apache.spark.sql.hive.execution.InsertIntoHiveTable import org.apache.spark.sql.hive.rapids.{GpuHiveTextFileFormat, GpuSaveAsHiveFile, RapidsHiveErrors} +import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim import org.apache.spark.sql.vectorized.ColumnarBatch final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, @@ -192,7 +193,7 @@ case class GpuInsertIntoHiveTable( // Report error if any static partition appears after a dynamic partition val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) if (isDynamic.init.zip(isDynamic.tail).contains((true, false))) { - throw new AnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_3079", Map.empty) } } diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala new file mode 100644 index 00000000000..789f7532d7c --- /dev/null +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala @@ -0,0 +1,114 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "311"} +{"spark": "312"} +{"spark": "313"} +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +spark-rapids-shim-json-lines ***/ + +package org.apache.spark.sql.rapids.shims + +import org.apache.spark.sql.AnalysisException + +object AnalysisExceptionShim { + + private def getMessage(errorClass: String, params: Map[String, String]): String = { + errorClass match { + case "_LEGACY_ERROR_TEMP_1137" => s"Unable to resolve ${params("name")} given " + + s"[${params("outputStr")}]" + case "_LEGACY_ERROR_TEMP_1128" => + s"Failed to resolve the schema for ${params("format")} for the partition column: " + + s"${params("partitionColumn")}. It must be specified manually." + case "UNABLE_TO_INFER_SCHEMA" => + s"Unable to infer schema for ${params("format")}. It must be specified manually." + case "_LEGACY_ERROR_TEMP_1132" => + s"A schema needs to be specified when using ${params("className")}." + case "_LEGACY_ERROR_TEMP_1133" => + "The user-specified schema doesn't match the actual schema: " + + s"user-specified: ${params("schema")}, actual: ${params("actualSchema")}. If " + + "you're using DataFrameReader.schema API or creating a table, please do not " + + "specify the schema. Or if you're scanning an existed table, please drop " + + "it and re-create it." + case "_LEGACY_ERROR_TEMP_1134" => + s"Unable to infer schema for ${params("format")} at ${params("fileCatalog")}. " + + "It must be specified manually" + case "_LEGACY_ERROR_TEMP_1135" => + s"${params("className")} is not a valid Spark SQL Data Source." + case "_LEGACY_ERROR_TEMP_1138" => + s"Hive built-in ORC data source must be used with Hive support enabled. " + + s"Please use the native ORC data source by setting 'spark.sql.orc.impl' to 'native'." + case "_LEGACY_ERROR_TEMP_1139" => + s"Failed to find data source: ${params("provider")}. Avro is built-in but external data " + + "source module since Spark 2.4. Please deploy the application as per " + + "the deployment section of \"Apache Avro Data Source Guide\"." + case "_LEGACY_ERROR_TEMP_1140" => + s"Failed to find data source: ${params("provider")}. Please deploy the application as " + + "per the deployment section of " + + "\"Structured Streaming + Kafka Integration Guide\"." + case "_LEGACY_ERROR_TEMP_1141" => + s"Multiple sources found for ${params("provider")} " + + s"(${params("sourceNames")}), please specify the fully qualified class name." + case "PATH_NOT_FOUND" => + s"Path does not exist: ${params("path")}" + case "_LEGACY_ERROR_TEMP_3079" => + "Dynamic partition cannot be the parent of a static partition." + case "_LEGACY_ERROR_TEMP_1288" => + s"Table ${params("tableName")} already exists. You need to drop it first." + case "_LEGACY_ERROR_TEMP_1241" => + s"CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory " + + s"${params("tablePath")}. To allow overwriting the existing non-empty directory, " + + s"set '${params("config")}' to true." + case "_LEGACY_ERROR_TEMP_1172" => + s"Parquet type not yet supported: ${params("parquetType")}." + case "_LEGACY_ERROR_TEMP_1173" => + s"Illegal Parquet type: ${params("parquetType")}." + case _ => + throw new IllegalStateException(s"Invalid errorClass in ${this.getClass.getSimpleName}") + } + } + + def throwException( + errorClass: String, + messageParameters: Map[String, String]) = + throw new AnalysisException(getMessage(errorClass, messageParameters)) + + + def throwException(msg: String) = throw new AnalysisException(msg) + +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index c3152a8a235..ad2b09d72f3 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -29,7 +29,7 @@ import org.apache.parquet.schema.LogicalTypeAnnotation._ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim import org.apache.spark.sql.types._ object ParquetSchemaClipShims { @@ -67,10 +67,12 @@ object ParquetSchemaClipShims { if (typeAnnotation == null) s"$typeName" else s"$typeName ($typeAnnotation)" def typeNotImplemented() = - TrampolineUtil.throwAnalysisException(s"Parquet type not yet supported: $typeString") + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1172", + Map("parquetType" -> s"$typeString")) def illegalType() = - TrampolineUtil.throwAnalysisException(s"Illegal Parquet type: $typeString") + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1173", + Map("parquetType" -> s"$typeString")) // When maxPrecision = -1, we skip precision range check, and always respect the precision // specified in field.getDecimalMetadata. This is useful when interpreting decimal types stored @@ -82,7 +84,7 @@ object ParquetSchemaClipShims { val scale = decimalLogicalTypeAnnotation.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - TrampolineUtil.throwAnalysisException( + AnalysisExceptionShim.throwException( s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -143,14 +145,14 @@ object ParquetSchemaClipShims { TimestampType case timestamp: TimestampLogicalTypeAnnotation if timestamp.getUnit == TimeUnit.NANOS && ParquetLegacyNanoAsLongShims.legacyParquetNanosAsLong => - TrampolineUtil.throwAnalysisException( + AnalysisExceptionShim.throwException( "GPU does not support spark.sql.legacy.parquet.nanosAsLong") case _ => illegalType() } case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - TrampolineUtil.throwAnalysisException( + AnalysisExceptionShim.throwException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index 56708017a23..88717c84821 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -44,7 +44,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport.containsFieldIds import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim import org.apache.spark.sql.types._ object ParquetSchemaClipShims { @@ -109,10 +109,12 @@ object ParquetSchemaClipShims { if (typeAnnotation == null) s"$typeName" else s"$typeName ($typeAnnotation)" def typeNotImplemented() = - TrampolineUtil.throwAnalysisException(s"Parquet type not yet supported: $typeString") + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1172", + Map("parquetType" -> s"$typeString")) def illegalType() = - TrampolineUtil.throwAnalysisException(s"Illegal Parquet type: $parquetType") + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1173", + Map("parquetType" -> s"$typeString")) // When maxPrecision = -1, we skip precision range check, and always respect the precision // specified in field.getDecimalMetadata. This is useful when interpreting decimal types stored @@ -124,7 +126,7 @@ object ParquetSchemaClipShims { val scale = decimalLogicalTypeAnnotation.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - TrampolineUtil.throwAnalysisException(s"Invalid decimal precision: $typeName " + + AnalysisExceptionShim.throwException(s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -183,14 +185,14 @@ object ParquetSchemaClipShims { ParquetTimestampAnnotationShims.timestampTypeForMillisOrMicros(timestamp) case timestamp: TimestampLogicalTypeAnnotation if timestamp.getUnit == TimeUnit.NANOS && ParquetLegacyNanoAsLongShims.legacyParquetNanosAsLong => - TrampolineUtil.throwAnalysisException( + AnalysisExceptionShim.throwException( "GPU does not support spark.sql.legacy.parquet.nanosAsLong") case _ => illegalType() } case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - TrampolineUtil.throwAnalysisException( + AnalysisExceptionShim.throwException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } 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 095301c238f..5c8fe260610 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 @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.spark.SparkException -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, ExternalCatalog, ExternalCatalogUtils, ExternalCatalogWithListener} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -48,6 +48,7 @@ import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.execution.InsertIntoHiveTable import org.apache.spark.sql.hive.rapids.{GpuHiveTextFileFormat, GpuSaveAsHiveFile, RapidsHiveErrors} +import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim import org.apache.spark.sql.vectorized.ColumnarBatch final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, @@ -181,7 +182,7 @@ case class GpuInsertIntoHiveTable( // Report error if any static partition appears after a dynamic partition val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) if (isDynamic.init.zip(isDynamic.tail).contains((true, false))) { - throw new AnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) {} + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_3079", Map.empty) } } diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index 0e1e88a2b8f..8c43d54a724 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -42,7 +42,7 @@ import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils} import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeSet, Expression, SortOrder} @@ -51,6 +51,7 @@ import org.apache.spark.sql.connector.write.WriterCommitMessage import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.datasources.{GpuWriteFiles, GpuWriteFilesExec, GpuWriteFilesSpec, WriteTaskResult, WriteTaskStats} import org.apache.spark.sql.execution.datasources.FileFormatWriter.OutputSpec +import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -61,8 +62,8 @@ object GpuFileFormatWriter extends Logging { private def verifySchema(format: ColumnarFileFormat, schema: StructType): Unit = { schema.foreach { field => if (!format.supportDataType(field.dataType)) { - throw new AnalysisException( - s"$format data source does not support ${field.dataType.catalogString} data type.") {} + AnalysisExceptionShim.throwException( + s"$format data source does not support ${field.dataType.catalogString} data type.") } } } diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala index 7a514d35b76..4d2ef17b224 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala @@ -64,8 +64,8 @@ case class GpuCreateDataSourceTableAsSelectCommand( s"Expect the table $tableName has been dropped when the save mode is Overwrite") if (mode == SaveMode.ErrorIfExists) { - throw new AnalysisException(s"Table $tableName already exists. " + - "You need to drop it first.") {} + AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1288", + Map("tableName" -> s"$tableName")) } if (mode == SaveMode.Ignore) { // Since the table already exists and the save mode is Ignore, we will just return. diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala new file mode 100644 index 00000000000..09b42dfd7a7 --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ + +package org.apache.spark.sql.rapids.shims + +import org.apache.spark.sql.AnalysisException + +object AnalysisExceptionShim { + + class RapidsAnalysisException(msg: String) extends AnalysisException(msg) + + def throwException( + errorClass: String, + messageParameters: Map[String, String]) = + throw new AnalysisException(errorClass, messageParameters) + + def throwException(msg: String) = throw new RapidsAnalysisException(msg) + +} From 0f765c9b3910aa3c174de709850d9a5fbc4aea89 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 19 Apr 2024 15:45:00 -0700 Subject: [PATCH 03/15] POM changes Signed-off-by: Raza Jafri --- build/buildall | 2 +- pom.xml | 29 +++++++++++++++++++++++++++-- scala2.13/pom.xml | 31 ++++++++++++++++++++++++++++--- scala2.13/sql-plugin/pom.xml | 20 ++++++++++++++++++++ sql-plugin/pom.xml | 20 ++++++++++++++++++++ 5 files changed, 96 insertions(+), 6 deletions(-) diff --git a/build/buildall b/build/buildall index e8c0610deb7..cd86c1414cb 100755 --- a/build/buildall +++ b/build/buildall @@ -196,7 +196,7 @@ case $DIST_PROFILE in SPARK_SHIM_VERSIONS=($(versionsFromDistProfile "minimumFeatureVersionMix")) ;; - 3*) + [34]*) <<< $DIST_PROFILE IFS="," read -ra SPARK_SHIM_VERSIONS INCLUDED_BUILDVERS_OPT="-Dincluded_buildvers=$DIST_PROFILE" unset DIST_PROFILE diff --git a/pom.xml b/pom.xml index c939f8d5891..5169978ac2a 100644 --- a/pom.xml +++ b/pom.xml @@ -611,6 +611,26 @@ delta-lake/delta-stub + + release400 + + + buildver + 400 + + + + 400 + ${spark400.version} + ${spark400.version} + 1.13.1 + ${spark330.iceberg.version} + 2.0.7 + + + delta-lake/delta-stub + + source-javadoc @@ -642,7 +662,7 @@ scala-2.13 2.13 - 2.13.8 + 2.13.13 @@ -782,6 +802,7 @@ 3.4.1-databricks 3.5.0 3.5.1 + 4.0.0-SNAPSHOT 3.12.4 4.3.0 3.1.1 @@ -835,6 +856,9 @@ 351 + 330db, @@ -890,6 +914,7 @@ 351 + 400 ${noSnapshotScala213.buildvers} @@ -1502,7 +1527,7 @@ This will force full Scala code rebuild in downstream modules. Unexpected buildver value ${buildver} for a Scala 2.13 build, only Apache Spark versions 3.3.0 (330) and higher are supported, no vendor builds such as 330db buildver - [3-9][3-9][0-9] + (?:[3-9][3-9]|[4-9][0-9])[0-9] --> diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index cf3bfb48373..a790e460719 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -611,6 +611,26 @@ delta-lake/delta-stub + + release400 + + + buildver + 400 + + + + 400 + ${spark400.version} + ${spark400.version} + 1.13.1 + ${spark330.iceberg.version} + 2.0.7 + + + delta-lake/delta-stub + + source-javadoc @@ -642,7 +662,7 @@ scala-2.13 2.13 - 2.13.8 + 2.13.13 @@ -724,7 +744,7 @@ 2.13 2.8.0 incremental - 2.13.8 + 2.13.13 + 400 + 330db, @@ -890,6 +914,7 @@ 351 + 400 ${noSnapshotScala213.buildvers} @@ -1502,7 +1527,7 @@ This will force full Scala code rebuild in downstream modules. Unexpected buildver value ${buildver} for a Scala 2.13 build, only Apache Spark versions 3.3.0 (330) and higher are supported, no vendor builds such as 330db buildver - [3-9][3-9][0-9] + (?:[3-9][3-9]|[4-9][0-9])[0-9] diff --git a/scala2.13/sql-plugin/pom.xml b/scala2.13/sql-plugin/pom.xml index 5bee34752c7..8d92bd43c0f 100644 --- a/scala2.13/sql-plugin/pom.xml +++ b/scala2.13/sql-plugin/pom.xml @@ -103,6 +103,26 @@ test + + + + release400 + + + buildver + 400 + + + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${spark.version} + + + + + diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index cbe7d873dff..6ee699f181f 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -103,6 +103,26 @@ test + + + + release400 + + + buildver + 400 + + + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${spark.version} + + + + + From a534c5a64f627b23d87f2a27038e046dc38350da Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Wed, 22 May 2024 16:27:42 -0700 Subject: [PATCH 04/15] Reuse the RapidsErrorUtils to throw the AnalysisException --- .../rapids/GpuDataWritingCommandExec.scala | 7 +- .../spark/rapids/GpuRunnableCommandExec.scala | 7 +- .../sql/hive/rapids/RapidsHiveErrors.scala | 7 +- .../spark/sql/rapids/GpuDataSourceBase.scala | 41 ++---- .../rapids/shims/ParquetSchemaClipShims.scala | 9 +- .../rapids/shims/GpuInsertIntoHiveTable.scala | 4 +- .../rapids/shims/AnalysisExceptionShim.scala | 114 -------------- .../sql/rapids/shims/RapidsErrorUtils.scala | 2 +- .../rapids/shims/RapidsQueryErrorUtils.scala | 139 ++++++++++++++++++ .../rapids/shims/ParquetSchemaClipShims.scala | 6 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 2 +- .../rapids/shims/ParquetSchemaClipShims.scala | 9 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 2 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 2 +- .../rapids/shims/GpuInsertIntoHiveTable.scala | 4 +- ...eDataSourceTableAsSelectCommandShims.scala | 3 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 2 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 2 +- .../rapids/shims/AnalysisExceptionShim.scala | 36 ----- .../rapids/shims/RapidsQueryErrorUtils.scala | 101 +++++++++++++ 20 files changed, 286 insertions(+), 213 deletions(-) delete mode 100644 sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala create mode 100644 sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala delete mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala create mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala index 3452b976a94..47f69967684 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.command.DataWritingCommand import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.GpuWriteJobStatsTracker -import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration @@ -84,8 +84,9 @@ object GpuDataWritingCommand { if (fs.exists(filePath) && fs.getFileStatus(filePath).isDirectory && fs.listStatus(filePath).length != 0) { - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1241", - Map("tablePath" -> s"$tablePath", "config" -> s"$allowNonEmptyLocationInCTASKey")) + throw RapidsErrorUtils. + createTableAsSelectWithNonEmptyDirectoryError(tablePath.toString, + allowNonEmptyLocationInCTASKey) } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala index c2d392fb473..ed26a254dda 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.GpuWriteJobStatsTracker -import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration @@ -82,8 +82,9 @@ object GpuRunnableCommand { if (fs.exists(filePath) && fs.getFileStatus(filePath).isDirectory && fs.listStatus(filePath).length != 0) { - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1241", - Map("tablePath" -> s"$tablePath", "config" -> s"$allowNonEmptyLocationInCTASKey")) + throw RapidsErrorUtils. + createTableAsSelectWithNonEmptyDirectoryError(tablePath.toString, + allowNonEmptyLocationInCTASKey) } } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala index 9ac7aafd98a..3548fbda906 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala @@ -17,11 +17,10 @@ package org.apache.spark.sql.hive.rapids import org.apache.hadoop.fs.Path - import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types.{DataType, DoubleType, FloatType, StringType} object RapidsHiveErrors { @@ -53,9 +52,7 @@ object RapidsHiveErrors { } def cannotResolveAttributeError(name: String, outputStr: String): Throwable = { - AnalysisExceptionShim.throwException( - errorClass = "_LEGACY_ERROR_TEMP_1137", - Map("name" -> s"$name", "outputStr" -> s"$outputStr")) + throw RapidsErrorUtils.cannotResolveAttributeError(name, outputStr) } def writePartitionExceedConfigSizeWhenDynamicPartitionError( diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala index 9ebd16fa008..5589bca0435 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.shims.{AnalysisExceptionShim, SchemaUtilsShims} +import org.apache.spark.sql.rapids.shims.{RapidsErrorUtils, SchemaUtilsShims} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.{HadoopFSUtils, ThreadUtils, Utils} @@ -144,8 +144,8 @@ abstract class GpuDataSourceBase( } inferredOpt }.getOrElse { - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1128", - Map("format" -> s"$format", "partitionColumn" -> s"$partitionColumn")) + throw RapidsErrorUtils. + partitionColumnNotSpecifiedError(format.toString, partitionColumn) } } StructType(partitionFields) @@ -162,8 +162,7 @@ abstract class GpuDataSourceBase( caseInsensitiveOptions - "path", SparkShimImpl.filesFromFileIndex(tempFileIndex)) }.getOrElse { - AnalysisExceptionShim. - throwException("UNABLE_TO_INFER_SCHEMA", Map("format" -> s"$format")) + throw RapidsErrorUtils.dataSchemaNotSpecifiedError(format.toString) } // We just print a waring message if the data schema and partition schema have the duplicate @@ -201,14 +200,13 @@ abstract class GpuDataSourceBase( case (dataSource: RelationProvider, None) => dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) case (_: SchemaRelationProvider, None) => - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1132", - Map("className" -> s"$className")) + throw RapidsErrorUtils.schemaNotSpecifiedForSchemaRelationProviderError(className) case (dataSource: RelationProvider, Some(schema)) => val baseRelation = dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) if (!DataType.equalsIgnoreCompatibleNullability(baseRelation.schema, schema)) { - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1133", - Map("schema" -> s"${schema.toDDL}", "actualSchema" -> s"${baseRelation.schema.toDDL}")) + throw RapidsErrorUtils.userSpecifiedSchemaMismatchActualSchemaError(schema, + baseRelation.schema) } baseRelation @@ -230,9 +228,8 @@ abstract class GpuDataSourceBase( caseInsensitiveOptions - "path", SparkShimImpl.filesFromFileIndex(fileCatalog)) }.getOrElse { - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1134", - Map("format" -> s"$format", - "fileCatalog" -> s"${fileCatalog.allFiles().mkString(",")}")) + throw RapidsErrorUtils. + dataSchemaNotSpecifiedError(format.toString, fileCatalog.allFiles().mkString(",")) } HadoopFsRelation( @@ -273,8 +270,7 @@ abstract class GpuDataSourceBase( caseInsensitiveOptions)(sparkSession) case _ => - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1135", - Map("className" -> s"$className")) + throw RapidsErrorUtils.invalidDataSourceError(className) } relation match { @@ -408,16 +404,13 @@ object GpuDataSourceBase extends Logging { dataSource case Failure(error) => if (provider1.startsWith("org.apache.spark.sql.hive.orc")) { - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1138", - Map.empty[String, String]) + throw RapidsErrorUtils.orcNotUsedWithHiveEnabledError() } else if (provider1.toLowerCase(Locale.ROOT) == "avro" || provider1 == "com.databricks.spark.avro" || provider1 == "org.apache.spark.sql.avro") { - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1139", - Map("provider" -> s"$provider1")) + throw RapidsErrorUtils.failedToFindAvroDataSourceError(provider1) } else if (provider1.toLowerCase(Locale.ROOT) == "kafka") { - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1140", - Map("provider" -> s"$provider1")) + throw RapidsErrorUtils.failedToFindKafkaDataSourceError(provider1) } else { throw new ClassNotFoundException( s"Failed to find data source: $provider1. Please find packages at " + @@ -450,9 +443,7 @@ object GpuDataSourceBase extends Logging { s"defaulting to the internal datasource (${internalSources.head.getClass.getName}).") internalSources.head.getClass } else { - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1141", - Map("provider" -> s"$provider1", - "sourceNames" -> s"${sourceNames.mkString(", ")}")) + throw RapidsErrorUtils.findMultipleDataSourceError(provider1, sourceNames) } } } catch { @@ -505,7 +496,7 @@ object GpuDataSourceBase extends Logging { } if (checkEmptyGlobPath && globResult.isEmpty) { - AnalysisExceptionShim.throwException("PATH_NOT_FOUND", Map("path" -> s"$globPath")) + throw RapidsErrorUtils.dataPathNotExistError(globPath.toString) } globResult @@ -519,7 +510,7 @@ object GpuDataSourceBase extends Logging { ThreadUtils.parmap(nonGlobPaths, "checkPathsExist", numThreads) { path => val fs = path.getFileSystem(hadoopConf) if (!fs.exists(path)) { - AnalysisExceptionShim.throwException("PATH_NOT_FOUND", Map("path" -> s"$path")) + throw RapidsErrorUtils.dataPathNotExistError(path.toString) } } } catch { diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index 6195e70eb23..de219bcd67a 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -24,9 +24,8 @@ package com.nvidia.spark.rapids.shims import org.apache.parquet.schema._ import org.apache.parquet.schema.OriginalType._ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ - import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim +import org.apache.spark.sql.rapids.shims.{AnalysisExceptionShim, RapidsErrorUtils} import org.apache.spark.sql.types._ object ParquetSchemaClipShims { @@ -67,12 +66,10 @@ object ParquetSchemaClipShims { AnalysisExceptionShim.throwException(s"Parquet type not supported: $typeString") def typeNotImplemented() = - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1172", - Map("parquetType" -> s"$typeString")) + throw RapidsErrorUtils.parquetTypeUnsupportedYetError(typeString) def illegalType() = - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1173", - Map("parquetType" -> s"$typeString")) + throw RapidsErrorUtils.illegalParquetTypeError(typeString) // When maxPrecision = -1, we skip precision range check, and always respect the precision // specified in field.getDecimalMetadata. This is useful when interpreting decimal types stored 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 ca8a83c5930..5a935ea21d6 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 @@ -58,7 +58,7 @@ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.client.hive._ import org.apache.spark.sql.hive.execution.InsertIntoHiveTable import org.apache.spark.sql.hive.rapids.{GpuHiveTextFileFormat, GpuSaveAsHiveFile, RapidsHiveErrors} -import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.vectorized.ColumnarBatch final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, @@ -193,7 +193,7 @@ case class GpuInsertIntoHiveTable( // Report error if any static partition appears after a dynamic partition val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) if (isDynamic.init.zip(isDynamic.tail).contains((true, false))) { - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_3079", Map.empty) + throw RapidsErrorUtils.dynamicPartitionParentError } } diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala deleted file mode 100644 index 789f7532d7c..00000000000 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Copyright (c) 2024, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/*** spark-rapids-shim-json-lines -{"spark": "311"} -{"spark": "312"} -{"spark": "313"} -{"spark": "320"} -{"spark": "321"} -{"spark": "321cdh"} -{"spark": "322"} -{"spark": "323"} -{"spark": "324"} -{"spark": "330"} -{"spark": "330cdh"} -{"spark": "330db"} -{"spark": "331"} -{"spark": "332"} -{"spark": "332cdh"} -{"spark": "332db"} -{"spark": "333"} -{"spark": "334"} -{"spark": "340"} -{"spark": "341"} -{"spark": "341db"} -{"spark": "342"} -{"spark": "343"} -{"spark": "350"} -{"spark": "351"} -spark-rapids-shim-json-lines ***/ - -package org.apache.spark.sql.rapids.shims - -import org.apache.spark.sql.AnalysisException - -object AnalysisExceptionShim { - - private def getMessage(errorClass: String, params: Map[String, String]): String = { - errorClass match { - case "_LEGACY_ERROR_TEMP_1137" => s"Unable to resolve ${params("name")} given " + - s"[${params("outputStr")}]" - case "_LEGACY_ERROR_TEMP_1128" => - s"Failed to resolve the schema for ${params("format")} for the partition column: " + - s"${params("partitionColumn")}. It must be specified manually." - case "UNABLE_TO_INFER_SCHEMA" => - s"Unable to infer schema for ${params("format")}. It must be specified manually." - case "_LEGACY_ERROR_TEMP_1132" => - s"A schema needs to be specified when using ${params("className")}." - case "_LEGACY_ERROR_TEMP_1133" => - "The user-specified schema doesn't match the actual schema: " + - s"user-specified: ${params("schema")}, actual: ${params("actualSchema")}. If " + - "you're using DataFrameReader.schema API or creating a table, please do not " + - "specify the schema. Or if you're scanning an existed table, please drop " + - "it and re-create it." - case "_LEGACY_ERROR_TEMP_1134" => - s"Unable to infer schema for ${params("format")} at ${params("fileCatalog")}. " + - "It must be specified manually" - case "_LEGACY_ERROR_TEMP_1135" => - s"${params("className")} is not a valid Spark SQL Data Source." - case "_LEGACY_ERROR_TEMP_1138" => - s"Hive built-in ORC data source must be used with Hive support enabled. " + - s"Please use the native ORC data source by setting 'spark.sql.orc.impl' to 'native'." - case "_LEGACY_ERROR_TEMP_1139" => - s"Failed to find data source: ${params("provider")}. Avro is built-in but external data " + - "source module since Spark 2.4. Please deploy the application as per " + - "the deployment section of \"Apache Avro Data Source Guide\"." - case "_LEGACY_ERROR_TEMP_1140" => - s"Failed to find data source: ${params("provider")}. Please deploy the application as " + - "per the deployment section of " + - "\"Structured Streaming + Kafka Integration Guide\"." - case "_LEGACY_ERROR_TEMP_1141" => - s"Multiple sources found for ${params("provider")} " + - s"(${params("sourceNames")}), please specify the fully qualified class name." - case "PATH_NOT_FOUND" => - s"Path does not exist: ${params("path")}" - case "_LEGACY_ERROR_TEMP_3079" => - "Dynamic partition cannot be the parent of a static partition." - case "_LEGACY_ERROR_TEMP_1288" => - s"Table ${params("tableName")} already exists. You need to drop it first." - case "_LEGACY_ERROR_TEMP_1241" => - s"CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory " + - s"${params("tablePath")}. To allow overwriting the existing non-empty directory, " + - s"set '${params("config")}' to true." - case "_LEGACY_ERROR_TEMP_1172" => - s"Parquet type not yet supported: ${params("parquetType")}." - case "_LEGACY_ERROR_TEMP_1173" => - s"Illegal Parquet type: ${params("parquetType")}." - case _ => - throw new IllegalStateException(s"Invalid errorClass in ${this.getClass.getSimpleName}") - } - } - - def throwException( - errorClass: String, - messageParameters: Map[String, String]) = - throw new AnalysisException(getMessage(errorClass, messageParameters)) - - - def throwException(msg: String) = throw new AnalysisException(msg) - -} \ No newline at end of file diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index f23229e0956..ade893e1fe7 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} -object RapidsErrorUtils { +object RapidsErrorUtils extends RapidsQueryErrorUtils { def invalidArrayIndexError(index: Int, numElements: Int, isElementAtF: Boolean = false): ArrayIndexOutOfBoundsException = { // Follow the Spark string format before 3.3.0 diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala new file mode 100644 index 00000000000..b119b9566bb --- /dev/null +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -0,0 +1,139 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "311"} +{"spark": "312"} +{"spark": "313"} +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +spark-rapids-shim-json-lines ***/ + +package org.apache.spark.sql.rapids.shims + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.types.StructType + +trait RapidsQueryErrorUtils { + + def createTableAsSelectWithNonEmptyDirectoryError(tablePath: String, conf: String): Throwable = { + new AnalysisException(s"CREATE-TABLE-AS-SELECT cannot create table with location to a " + + s"non-empty directory $tablePath. To allow overwriting the existing non-empty directory, " + + s"set '$conf' to true.") + } + + def cannotResolveAttributeError(name: String, outputStr: String): Throwable = { + new AnalysisException(s"Unable to resolve $name given [$outputStr]") + } + + def partitionColumnNotSpecifiedError(format: String, partitionColumn: String): Throwable = { + new AnalysisException(s"Failed to resolve the schema for $format for the partition column: " + + s"$partitionColumn. It must be specified manually.") + } + + def dataSchemaNotSpecifiedError(format: String): Throwable = { + new AnalysisException(s"Unable to infer schema for $format. It must be specified manually.") + } + + def schemaNotSpecifiedForSchemaRelationProviderError(className: String): Throwable = { + new AnalysisException(s"A schema needs to be specified when using $className.") + } + + def userSpecifiedSchemaMismatchActualSchemaError( + schema: StructType, + actualSchema: StructType): Throwable = { + new AnalysisException("The user-specified schema doesn't match the actual schema: " + + s"user-specified: ${schema.toDDL}, actual: ${actualSchema.toDDL}. If " + + "you're using DataFrameReader.schema API or creating a table, please do not " + + "specify the schema. Or if you're scanning an existed table, please drop " + + "it and re-create it.") + } + + def dataSchemaNotSpecifiedError(format: String, fileCatalog: String): Throwable = { + new AnalysisException(s"Unable to infer schema for $format at $fileCatalog. " + + "It must be specified manually") + } + + def invalidDataSourceError(className: String): Throwable = { + new AnalysisException(s"$className is not a valid Spark SQL Data Source.") + } + + def orcNotUsedWithHiveEnabledError(): Throwable = { + new AnalysisException( + s"Hive built-in ORC data source must be used with Hive support enabled. " + + s"Please use the native ORC data source by setting 'spark.sql.orc.impl' to 'native'.") + } + + def failedToFindAvroDataSourceError(provider: String): Throwable = { + new AnalysisException( + s"Failed to find data source: $provider. Avro is built-in but external data " + + "source module since Spark 2.4. Please deploy the application as per " + + "the deployment section of \"Apache Avro Data Source Guide\".") + } + + def failedToFindKafkaDataSourceError(provider: String): Throwable = { + new AnalysisException( + s"Failed to find data source: $provider. Please deploy the application as " + + "per the deployment section of " + + "\"Structured Streaming + Kafka Integration Guide\".") + } + + def findMultipleDataSourceError(provider: String, sourceNames: Seq[String]): Throwable = { + new AnalysisException( + s"Multiple sources found for $provider " + + s"(${sourceNames.mkString(", ")}), please specify the fully qualified class name.") + } + + def dataPathNotExistError(path: String): Throwable = { + new AnalysisException(s"Path does not exist: $path") + } + + def dynamicPartitionParentError: Throwable = { + new AnalysisException("Dynamic partition cannot be the parent of a static partition.") + } + + def tableOrViewAlreadyExistsError(tableName: String): Throwable = { + new AnalysisException(s"Table $tableName already exists. You need to drop it first.") + } + + def parquetTypeUnsupportedYetError(parquetType: String): Throwable = { + new AnalysisException(s"Parquet type not yet supported: $parquetType.") + } + + def illegalParquetTypeError(parquetType: String): Throwable = { + new AnalysisException(s"Illegal Parquet type: $parquetType.") + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index ad2b09d72f3..adde6e24bc5 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -67,12 +67,10 @@ object ParquetSchemaClipShims { if (typeAnnotation == null) s"$typeName" else s"$typeName ($typeAnnotation)" def typeNotImplemented() = - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1172", - Map("parquetType" -> s"$typeString")) + throw RapidsErrorUtils.parquetTypeUnsupportedYetError(typeString) def illegalType() = - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1173", - Map("parquetType" -> s"$typeString")) + throw RapidsErrorUtils.illegalParquetTypeError(typeString) // When maxPrecision = -1, we skip precision range check, and always respect the precision // specified in field.getDecimalMetadata. This is useful when interpreting decimal types stored diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index b301397255a..e971acfe31d 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} -object RapidsErrorUtils { +object RapidsErrorUtils extends RapidsQueryErrorUtils { def invalidArrayIndexError(index: Int, numElements: Int, isElementAtF: Boolean = false): ArrayIndexOutOfBoundsException = { // Follow the Spark string format before 3.3.0 diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index 88717c84821..6109148b66b 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -44,7 +44,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport.containsFieldIds import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim +import org.apache.spark.sql.rapids.shims.{AnalysisExceptionShim, RapidsErrorUtils} import org.apache.spark.sql.types._ object ParquetSchemaClipShims { @@ -109,12 +109,11 @@ object ParquetSchemaClipShims { if (typeAnnotation == null) s"$typeName" else s"$typeName ($typeAnnotation)" def typeNotImplemented() = - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1172", - Map("parquetType" -> s"$typeString")) + throw RapidsErrorUtils.parquetTypeUnsupportedYetError(typeString) def illegalType() = - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1173", - Map("parquetType" -> s"$typeString")) + throw RapidsErrorUtils.illegalParquetTypeError(typeString) + // When maxPrecision = -1, we skip precision range check, and always respect the precision // specified in field.getDecimalMetadata. This is useful when interpreting decimal types stored diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index bb28c370749..93133eb5ae0 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} -object RapidsErrorUtils extends RapidsErrorUtilsFor330plus { +object RapidsErrorUtils extends RapidsErrorUtilsFor330plus with RapidsQueryErrorUtils { def mapKeyNotExistError( key: String, diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 1012b28d8b7..04a654da664 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -22,7 +22,7 @@ package org.apache.spark.sql.rapids.shims import org.apache.spark.sql.errors.QueryExecutionErrors -object RapidsErrorUtils extends RapidsErrorUtilsBase { +object RapidsErrorUtils extends RapidsErrorUtilsBase with RapidsQueryErrorUtils { def sqlArrayIndexNotStartAtOneError(): RuntimeException = { QueryExecutionErrors.elementAtByIndexZeroError(context = null) } 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 5c8fe260610..ebcad48b850 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 @@ -48,7 +48,7 @@ import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.execution.InsertIntoHiveTable import org.apache.spark.sql.hive.rapids.{GpuHiveTextFileFormat, GpuSaveAsHiveFile, RapidsHiveErrors} -import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim +import org.apache.spark.sql.rapids.shims.{AnalysisExceptionShim, RapidsErrorUtils} import org.apache.spark.sql.vectorized.ColumnarBatch final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, @@ -182,7 +182,7 @@ case class GpuInsertIntoHiveTable( // Report error if any static partition appears after a dynamic partition val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) if (isDynamic.init.zip(isDynamic.tail).contains((true, false))) { - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_3079", Map.empty) + throw RapidsErrorUtils.dynamicPartitionParentError } } diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala index 4d2ef17b224..6308f24c552 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala @@ -64,8 +64,7 @@ case class GpuCreateDataSourceTableAsSelectCommand( s"Expect the table $tableName has been dropped when the save mode is Overwrite") if (mode == SaveMode.ErrorIfExists) { - AnalysisExceptionShim.throwException("_LEGACY_ERROR_TEMP_1288", - Map("tableName" -> s"$tableName")) + throw RapidsErrorUtils.tableOrViewAlreadyExistsError(tableName) } if (mode == SaveMode.Ignore) { // Since the table already exists and the save mode is Ignore, we will just return. diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 8ee0485ab36..e6f8886f19c 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} -object RapidsErrorUtils extends RapidsErrorUtilsFor330plus { +object RapidsErrorUtils extends RapidsErrorUtilsFor330plus with RapidsQueryErrorUtils { def mapKeyNotExistError( key: String, diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index a0ba17f9bd4..71e3496412a 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -21,7 +21,7 @@ package org.apache.spark.sql.rapids.shims import org.apache.spark.sql.errors.QueryExecutionErrors -object RapidsErrorUtils extends RapidsErrorUtilsBase { +object RapidsErrorUtils extends RapidsErrorUtilsBase with RapidsQueryErrorUtils { def sqlArrayIndexNotStartAtOneError(): RuntimeException = { QueryExecutionErrors.invalidIndexOfZeroError(context = null) } diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala deleted file mode 100644 index 09b42dfd7a7..00000000000 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/AnalysisExceptionShim.scala +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Copyright (c) 2024, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/*** spark-rapids-shim-json-lines -{"spark": "400"} -spark-rapids-shim-json-lines ***/ - -package org.apache.spark.sql.rapids.shims - -import org.apache.spark.sql.AnalysisException - -object AnalysisExceptionShim { - - class RapidsAnalysisException(msg: String) extends AnalysisException(msg) - - def throwException( - errorClass: String, - messageParameters: Map[String, String]) = - throw new AnalysisException(errorClass, messageParameters) - - def throwException(msg: String) = throw new RapidsAnalysisException(msg) - -} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala new file mode 100644 index 00000000000..13aa099698a --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -0,0 +1,101 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ + +package org.apache.spark.sql.rapids.shims + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.types.StructType + +trait RapidsQueryErrorUtils { + + def createTableAsSelectWithNonEmptyDirectoryError(tablePath: String, conf: String): Throwable = { + QueryCompilationErrors.createTableAsSelectWithNonEmptyDirectoryError(tablePath) + } + + def cannotResolveAttributeError(name: String, outputStr: String): Throwable = { + QueryCompilationErrors.cannotResolveAttributeError(name, outputStr) + } + + def partitionColumnNotSpecifiedError(format: String, partitionColumn: String): Throwable = { + QueryCompilationErrors.partitionColumnNotSpecifiedError(format, partitionColumn) + } + + def dataSchemaNotSpecifiedError(format: String): Throwable = { + QueryCompilationErrors.dataSchemaNotSpecifiedError(format) + } + + def schemaNotSpecifiedForSchemaRelationProviderError(className: String): Throwable = { + QueryCompilationErrors.schemaNotSpecifiedForSchemaRelationProviderError(className) + } + + def userSpecifiedSchemaMismatchActualSchemaError( + schema: StructType, + actualSchema: StructType): Throwable = { + QueryCompilationErrors.userSpecifiedSchemaMismatchActualSchemaError(schema, actualSchema) + } + + def dataSchemaNotSpecifiedError(format: String, fileCatalog: String): Throwable = { + QueryCompilationErrors.dataSchemaNotSpecifiedError(format, fileCatalog) + } + + def invalidDataSourceError(className: String): Throwable = { + QueryCompilationErrors.invalidDataSourceError(className) + } + + def orcNotUsedWithHiveEnabledError(): Throwable = { + QueryCompilationErrors.orcNotUsedWithHiveEnabledError() + } + + def failedToFindAvroDataSourceError(provider: String): Throwable = { + QueryCompilationErrors.failedToFindAvroDataSourceError(provider) + } + + def failedToFindKafkaDataSourceError(provider: String): Throwable = { + QueryCompilationErrors.failedToFindKafkaDataSourceError(provider) + } + + def findMultipleDataSourceError(provider: String, sourceNames: Seq[String]): Throwable = { + QueryCompilationErrors.findMultipleDataSourceError(provider, sourceNames) + } + + def dataPathNotExistError(path: String): Throwable = { + QueryCompilationErrors.dataPathNotExistError(path) + } + + def dynamicPartitionParentError: Throwable = { + /** This exception doesn't have a helper method so the errorClass has to be hardcoded */ + throw new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_3079", + messageParameters = Map.empty) + } + + def tableOrViewAlreadyExistsError(tableName: String): Throwable = { + QueryCompilationErrors.tableOrViewAlreadyExistsError(tableName) + } + + def parquetTypeUnsupportedYetError(parquetType: String): Throwable = { + QueryCompilationErrors.parquetTypeUnsupportedYetError(parquetType) + } + + def illegalParquetTypeError(parquetType: String): Throwable = { + QueryCompilationErrors.illegalParquetTypeError(parquetType) + } +} \ No newline at end of file From 88a2673eca02886ab7c104b34c797797253f409c Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Wed, 22 May 2024 17:17:26 -0700 Subject: [PATCH 05/15] Revert "POM changes" This reverts commit 0f765c9b3910aa3c174de709850d9a5fbc4aea89. --- build/buildall | 2 +- pom.xml | 29 ++--------------------------- scala2.13/pom.xml | 31 +++---------------------------- scala2.13/sql-plugin/pom.xml | 20 -------------------- sql-plugin/pom.xml | 20 -------------------- 5 files changed, 6 insertions(+), 96 deletions(-) diff --git a/build/buildall b/build/buildall index cd86c1414cb..e8c0610deb7 100755 --- a/build/buildall +++ b/build/buildall @@ -196,7 +196,7 @@ case $DIST_PROFILE in SPARK_SHIM_VERSIONS=($(versionsFromDistProfile "minimumFeatureVersionMix")) ;; - [34]*) + 3*) <<< $DIST_PROFILE IFS="," read -ra SPARK_SHIM_VERSIONS INCLUDED_BUILDVERS_OPT="-Dincluded_buildvers=$DIST_PROFILE" unset DIST_PROFILE diff --git a/pom.xml b/pom.xml index 5169978ac2a..c939f8d5891 100644 --- a/pom.xml +++ b/pom.xml @@ -611,26 +611,6 @@ delta-lake/delta-stub - - release400 - - - buildver - 400 - - - - 400 - ${spark400.version} - ${spark400.version} - 1.13.1 - ${spark330.iceberg.version} - 2.0.7 - - - delta-lake/delta-stub - - source-javadoc @@ -662,7 +642,7 @@ scala-2.13 2.13 - 2.13.13 + 2.13.8 @@ -802,7 +782,6 @@ 3.4.1-databricks 3.5.0 3.5.1 - 4.0.0-SNAPSHOT 3.12.4 4.3.0 3.1.1 @@ -856,9 +835,6 @@ 351 - 330db, @@ -914,7 +890,6 @@ 351 - 400 ${noSnapshotScala213.buildvers} @@ -1527,7 +1502,7 @@ This will force full Scala code rebuild in downstream modules. Unexpected buildver value ${buildver} for a Scala 2.13 build, only Apache Spark versions 3.3.0 (330) and higher are supported, no vendor builds such as 330db buildver - (?:[3-9][3-9]|[4-9][0-9])[0-9] + [3-9][3-9][0-9] --> diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index a790e460719..cf3bfb48373 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -611,26 +611,6 @@ delta-lake/delta-stub - - release400 - - - buildver - 400 - - - - 400 - ${spark400.version} - ${spark400.version} - 1.13.1 - ${spark330.iceberg.version} - 2.0.7 - - - delta-lake/delta-stub - - source-javadoc @@ -662,7 +642,7 @@ scala-2.13 2.13 - 2.13.13 + 2.13.8 @@ -744,7 +724,7 @@ 2.13 2.8.0 incremental - 2.13.13 + 2.13.8 - 400 - 330db, @@ -914,7 +890,6 @@ 351 - 400 ${noSnapshotScala213.buildvers} @@ -1527,7 +1502,7 @@ This will force full Scala code rebuild in downstream modules. Unexpected buildver value ${buildver} for a Scala 2.13 build, only Apache Spark versions 3.3.0 (330) and higher are supported, no vendor builds such as 330db buildver - (?:[3-9][3-9]|[4-9][0-9])[0-9] + [3-9][3-9][0-9] diff --git a/scala2.13/sql-plugin/pom.xml b/scala2.13/sql-plugin/pom.xml index 8d92bd43c0f..5bee34752c7 100644 --- a/scala2.13/sql-plugin/pom.xml +++ b/scala2.13/sql-plugin/pom.xml @@ -103,26 +103,6 @@ test - - - - release400 - - - buildver - 400 - - - - - org.apache.spark - spark-catalyst_${scala.binary.version} - ${spark.version} - - - - - diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index 6ee699f181f..cbe7d873dff 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -103,26 +103,6 @@ test - - - - release400 - - - buildver - 400 - - - - - org.apache.spark - spark-catalyst_${scala.binary.version} - ${spark.version} - - - - - From 6bebb21c0e48e9419251e728aa8e9140180ad542 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Wed, 22 May 2024 17:22:28 -0700 Subject: [PATCH 06/15] Updated copyrights --- .../com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala | 2 +- .../scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala | 2 +- .../com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala | 2 +- .../org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala | 2 +- .../org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala | 2 +- .../org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala | 2 +- .../org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala | 2 +- .../org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala index 47f69967684..019f9b2e6b0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2023, NVIDIA CORPORATION. + * Copyright (c) 2019-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala index ed26a254dda..43bd593c0b5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index de219bcd67a..2269ff00e16 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index ade893e1fe7..7fa269db71a 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index e971acfe31d..68a6ce30569 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 93133eb5ae0..e5cdcd43568 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 04a654da664..7e58a54c921 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 71e3496412a..9b800d4e51a 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. From f967002d1b46b6338bef0a05632f62f9e9cfe960 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Wed, 22 May 2024 18:00:43 -0700 Subject: [PATCH 07/15] Added the TrampolineUtil method back to handle cases which don't use errorClass --- .../spark/sql/hive/rapids/RapidsHiveErrors.scala | 1 + .../rapids/GpuInsertIntoHadoopFsRelationCommand.scala | 5 +++-- .../catalyst/expressions/GpuRandomExpressions.scala | 4 ++-- .../spark/sql/rapids/execution/TrampolineUtil.scala | 7 ++++++- .../spark/rapids/shims/ParquetSchemaClipShims.scala | 10 ++++++---- .../spark/rapids/shims/ParquetSchemaClipShims.scala | 9 +++++---- .../spark/rapids/shims/ParquetSchemaClipShims.scala | 9 +++++---- .../spark/rapids/shims/GpuInsertIntoHiveTable.scala | 2 +- .../apache/spark/sql/rapids/GpuFileFormatWriter.scala | 4 ++-- 9 files changed, 31 insertions(+), 20 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala index 3548fbda906..40cac90680f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hive.rapids import org.apache.hadoop.fs.Path + import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Literal diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala index 9c55784e462..351249c2709 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala @@ -33,7 +33,8 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.{AlterTableAddPartitionCommand, AlterTableDropPartitionCommand, CommandUtils} import org.apache.spark.sql.execution.datasources.{FileFormatWriter, FileIndex, PartitioningUtils} import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode -import org.apache.spark.sql.rapids.shims.{AnalysisExceptionShim, SchemaUtilsShims} +import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.SchemaUtilsShims import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuInsertIntoHadoopFsRelationCommand( @@ -121,7 +122,7 @@ case class GpuInsertIntoHadoopFsRelationCommand( val pathExists = fs.exists(qualifiedOutputPath) (mode, pathExists) match { case (SaveMode.ErrorIfExists, true) => - AnalysisExceptionShim.throwException(s"path $qualifiedOutputPath already exists.") + TrampolineUtil.throwAnalysisException(s"path $qualifiedOutputPath already exists.") case (SaveMode.Overwrite, true) => if (ifPartitionNotExists && matchingPartitions.nonEmpty) { false diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala index e60bd0692df..1a2271f4194 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala @@ -24,7 +24,7 @@ import com.nvidia.spark.rapids.shims.ShimUnaryExpression import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionWithRandomSeed} -import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim +import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils @@ -52,7 +52,7 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres @transient protected lazy val seed: Long = child match { case GpuLiteral(s, IntegerType) => s.asInstanceOf[Int] case GpuLiteral(s, LongType) => s.asInstanceOf[Long] - case _ => AnalysisExceptionShim.throwException( + case _ => TrampolineUtil.throwAnalysisException( s"Input argument to $prettyName must be an integer, long or null literal.") } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala index cbd00177d7c..87e46ec16af 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala @@ -31,7 +31,7 @@ import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.scheduler.SparkListenerEvent import org.apache.spark.security.CryptoStreamUtils import org.apache.spark.serializer.{JavaSerializer, SerializerManager} -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode import org.apache.spark.sql.execution.SparkPlan @@ -156,6 +156,9 @@ object TrampolineUtil { TaskContext.get.taskMemoryManager() } + /** Throw a Spark analysis exception */ + def throwAnalysisException(msg: String) = throw new AnalysisException(msg) + /** Set the task context for the current thread */ def setTaskContext(tc: TaskContext): Unit = TaskContext.setTaskContext(tc) @@ -232,4 +235,6 @@ object TrampolineUtil { def postEvent(sc: SparkContext, sparkEvent: SparkListenerEvent): Unit = { sc.listenerBus.post(sparkEvent) } + + class RapidsAnalysisException(msg: String) extends AnalysisException(msg) } diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index 2269ff00e16..ed20762cc31 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -24,8 +24,10 @@ package com.nvidia.spark.rapids.shims import org.apache.parquet.schema._ import org.apache.parquet.schema.OriginalType._ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ + import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.shims.{AnalysisExceptionShim, RapidsErrorUtils} +import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ object ParquetSchemaClipShims { @@ -63,7 +65,7 @@ object ParquetSchemaClipShims { if (originalType == null) s"$typeName" else s"$typeName ($originalType)" def typeNotSupported() = - AnalysisExceptionShim.throwException(s"Parquet type not supported: $typeString") + TrampolineUtil.throwAnalysisException(s"Parquet type not supported: $typeString") def typeNotImplemented() = throw RapidsErrorUtils.parquetTypeUnsupportedYetError(typeString) @@ -79,7 +81,7 @@ object ParquetSchemaClipShims { val scale = field.getDecimalMetadata.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - AnalysisExceptionShim.throwException(s"Invalid decimal precision: $typeName " + + TrampolineUtil.throwAnalysisException(s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -119,7 +121,7 @@ object ParquetSchemaClipShims { case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - AnalysisExceptionShim.throwException( + TrampolineUtil.throwAnalysisException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index adde6e24bc5..41ee9527fbe 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -29,7 +29,8 @@ import org.apache.parquet.schema.LogicalTypeAnnotation._ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim +import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ object ParquetSchemaClipShims { @@ -82,7 +83,7 @@ object ParquetSchemaClipShims { val scale = decimalLogicalTypeAnnotation.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - AnalysisExceptionShim.throwException( + TrampolineUtil.throwAnalysisException( s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -143,14 +144,14 @@ object ParquetSchemaClipShims { TimestampType case timestamp: TimestampLogicalTypeAnnotation if timestamp.getUnit == TimeUnit.NANOS && ParquetLegacyNanoAsLongShims.legacyParquetNanosAsLong => - AnalysisExceptionShim.throwException( + TrampolineUtil.throwAnalysisException( "GPU does not support spark.sql.legacy.parquet.nanosAsLong") case _ => illegalType() } case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - AnalysisExceptionShim.throwException( + TrampolineUtil.throwAnalysisException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index 6109148b66b..fe3f8e34fc0 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -44,7 +44,8 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport.containsFieldIds import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.shims.{AnalysisExceptionShim, RapidsErrorUtils} +import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ object ParquetSchemaClipShims { @@ -125,7 +126,7 @@ object ParquetSchemaClipShims { val scale = decimalLogicalTypeAnnotation.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - AnalysisExceptionShim.throwException(s"Invalid decimal precision: $typeName " + + TrampolineUtil.throwAnalysisException(s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -184,14 +185,14 @@ object ParquetSchemaClipShims { ParquetTimestampAnnotationShims.timestampTypeForMillisOrMicros(timestamp) case timestamp: TimestampLogicalTypeAnnotation if timestamp.getUnit == TimeUnit.NANOS && ParquetLegacyNanoAsLongShims.legacyParquetNanosAsLong => - AnalysisExceptionShim.throwException( + TrampolineUtil.throwAnalysisException( "GPU does not support spark.sql.legacy.parquet.nanosAsLong") case _ => illegalType() } case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - AnalysisExceptionShim.throwException( + TrampolineUtil.throwAnalysisException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } 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 ebcad48b850..131141d3eac 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 @@ -48,7 +48,7 @@ import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.execution.InsertIntoHiveTable import org.apache.spark.sql.hive.rapids.{GpuHiveTextFileFormat, GpuSaveAsHiveFile, RapidsHiveErrors} -import org.apache.spark.sql.rapids.shims.{AnalysisExceptionShim, RapidsErrorUtils} +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.vectorized.ColumnarBatch final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index 8c43d54a724..da981c24123 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -51,7 +51,7 @@ import org.apache.spark.sql.connector.write.WriterCommitMessage import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.datasources.{GpuWriteFiles, GpuWriteFilesExec, GpuWriteFilesSpec, WriteTaskResult, WriteTaskStats} import org.apache.spark.sql.execution.datasources.FileFormatWriter.OutputSpec -import org.apache.spark.sql.rapids.shims.AnalysisExceptionShim +import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -62,7 +62,7 @@ object GpuFileFormatWriter extends Logging { private def verifySchema(format: ColumnarFileFormat, schema: StructType): Unit = { schema.foreach { field => if (!format.supportDataType(field.dataType)) { - AnalysisExceptionShim.throwException( + TrampolineUtil.throwAnalysisException( s"$format data source does not support ${field.dataType.catalogString} data type.") } } From d930fdf1213016915a365318f514c1e5b18a7e3b Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Tue, 28 May 2024 09:46:58 -0700 Subject: [PATCH 08/15] Add doc to the RapidsAnalysisException --- .../apache/spark/sql/rapids/execution/TrampolineUtil.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala index 87e46ec16af..ffa789085d4 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala @@ -236,5 +236,9 @@ object TrampolineUtil { sc.listenerBus.post(sparkEvent) } + /** + * This class is to only be used when there is no equivalent helper method in + * Spark to throw an AnalysisException. + */ class RapidsAnalysisException(msg: String) extends AnalysisException(msg) } From 915604767c3a8fd63f82f3f4e43ededd826af6ef Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 31 May 2024 15:49:43 -0700 Subject: [PATCH 09/15] addressed review comments --- ...GpuInsertIntoHadoopFsRelationCommand.scala | 5 +- .../sql/rapids/execution/TrampolineUtil.scala | 4 +- .../rapids/shims/RapidsQueryErrorUtils.scala | 28 ++---- .../rapids/shims/RapidsQueryErrorUtils.scala | 35 +++++-- .../sql/rapids/shims/RapidsErrorUtils.scala | 72 +------------- .../shims/RapidsErrorUtilsFor340Plus.scala | 95 +++++++++++++++++++ .../sql/rapids/shims/RapidsErrorUtils.scala | 31 ++++++ 7 files changed, 169 insertions(+), 101 deletions(-) rename sql-plugin/src/main/{spark400 => spark320}/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala (85%) create mode 100644 sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor340Plus.scala create mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala index 351249c2709..860a3aa7c2f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala @@ -33,8 +33,7 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.{AlterTableAddPartitionCommand, AlterTableDropPartitionCommand, CommandUtils} import org.apache.spark.sql.execution.datasources.{FileFormatWriter, FileIndex, PartitioningUtils} import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode -import org.apache.spark.sql.rapids.execution.TrampolineUtil -import org.apache.spark.sql.rapids.shims.SchemaUtilsShims +import org.apache.spark.sql.rapids.shims.{RapidsErrorUtils, SchemaUtilsShims} import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuInsertIntoHadoopFsRelationCommand( @@ -122,7 +121,7 @@ case class GpuInsertIntoHadoopFsRelationCommand( val pathExists = fs.exists(qualifiedOutputPath) (mode, pathExists) match { case (SaveMode.ErrorIfExists, true) => - TrampolineUtil.throwAnalysisException(s"path $qualifiedOutputPath already exists.") + throw RapidsErrorUtils.dataPathNotExistError(qualifiedOutputPath.toString) case (SaveMode.Overwrite, true) => if (ifPartitionNotExists && matchingPartitions.nonEmpty) { false diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala index 84d78d90c38..d79b5b046f1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala @@ -158,7 +158,7 @@ object TrampolineUtil { } /** Throw a Spark analysis exception */ - def throwAnalysisException(msg: String) = throw new AnalysisException(msg) + def throwAnalysisException(msg: String) = throw new RapidsAnalysisException(msg) /** Set the task context for the current thread */ def setTaskContext(tc: TaskContext): Unit = TaskContext.setTaskContext(tc) @@ -239,7 +239,7 @@ object TrampolineUtil { def postEvent(sc: SparkContext, sparkEvent: SparkListenerEvent): Unit = { sc.listenerBus.post(sparkEvent) } - + def getSparkHadoopUtilConf: Configuration = SparkHadoopUtil.get.conf /** diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala index b119b9566bb..79826586197 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -18,37 +18,21 @@ {"spark": "311"} {"spark": "312"} {"spark": "313"} -{"spark": "320"} -{"spark": "321"} -{"spark": "321cdh"} -{"spark": "322"} -{"spark": "323"} -{"spark": "324"} -{"spark": "330"} -{"spark": "330cdh"} -{"spark": "330db"} -{"spark": "331"} -{"spark": "332"} -{"spark": "332cdh"} -{"spark": "332db"} -{"spark": "333"} -{"spark": "334"} -{"spark": "340"} -{"spark": "341"} -{"spark": "341db"} -{"spark": "342"} -{"spark": "343"} -{"spark": "350"} -{"spark": "351"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims +import org.apache.hadoop.fs.Path + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.types.StructType trait RapidsQueryErrorUtils { + def outputPathAlreadyExistsError(qualifiedOutputPath: Path): Throwable = { + new AnalysisException(s"path $qualifiedOutputPath already exists.") + } + def createTableAsSelectWithNonEmptyDirectoryError(tablePath: String, conf: String): Throwable = { new AnalysisException(s"CREATE-TABLE-AS-SELECT cannot create table with location to a " + s"non-empty directory $tablePath. To allow overwriting the existing non-empty directory, " + diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala similarity index 85% rename from sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala rename to sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala index 13aa099698a..3b2d404c6d6 100644 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -15,17 +15,45 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims +import org.apache.hadoop.fs.Path + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types.StructType trait RapidsQueryErrorUtils { + def outputPathAlreadyExistsError(qualifiedOutputPath: Path): Throwable = { + QueryCompilationErrors.outputPathAlreadyExistsError(qualifiedOutputPath) + } + def createTableAsSelectWithNonEmptyDirectoryError(tablePath: String, conf: String): Throwable = { QueryCompilationErrors.createTableAsSelectWithNonEmptyDirectoryError(tablePath) } @@ -80,13 +108,6 @@ trait RapidsQueryErrorUtils { QueryCompilationErrors.dataPathNotExistError(path) } - def dynamicPartitionParentError: Throwable = { - /** This exception doesn't have a helper method so the errorClass has to be hardcoded */ - throw new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_3079", - messageParameters = Map.empty) - } - def tableOrViewAlreadyExistsError(tableName: String): Throwable = { QueryCompilationErrors.tableOrViewAlreadyExistsError(tableName) } diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index e6f8886f19c..c7d0c3be850 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -21,75 +21,13 @@ {"spark": "343"} {"spark": "350"} {"spark": "351"} -{"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims -import org.apache.spark.SparkDateTimeException -import org.apache.spark.sql.catalyst.trees.{Origin, SQLQueryContext} -import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} +import org.apache.spark.AnalysisException, SparkDateTimeException} -object RapidsErrorUtils extends RapidsErrorUtilsFor330plus with RapidsQueryErrorUtils { - - def mapKeyNotExistError( - key: String, - keyType: DataType, - origin: Origin): NoSuchElementException = { - throw new UnsupportedOperationException( - "`mapKeyNotExistError` has been removed since Spark 3.4.0. " - ) - } - - def invalidArrayIndexError( - index: Int, - numElements: Int, - isElementAtF: Boolean = false, - context: SQLQueryContext = null): ArrayIndexOutOfBoundsException = { - if (isElementAtF) { - QueryExecutionErrors.invalidElementAtIndexError(index, numElements, context) - } else { - QueryExecutionErrors.invalidArrayIndexError(index, numElements, context) - } - } - - def arithmeticOverflowError( - message: String, - hint: String = "", - errorContext: SQLQueryContext = null): ArithmeticException = { - QueryExecutionErrors.arithmeticOverflowError(message, hint, errorContext) - } - - def cannotChangeDecimalPrecisionError( - value: Decimal, - toType: DecimalType, - context: SQLQueryContext = null): ArithmeticException = { - QueryExecutionErrors.cannotChangeDecimalPrecisionError( - value, toType.precision, toType.scale, context - ) - } - - def overflowInIntegralDivideError(context: SQLQueryContext = null): ArithmeticException = { - QueryExecutionErrors.arithmeticOverflowError( - "Overflow in integral divide", "try_divide", context - ) - } - - def sparkDateTimeException(infOrNan: String): SparkDateTimeException = { - // These are the arguments required by SparkDateTimeException class to create error message. - val errorClass = "CAST_INVALID_INPUT" - val messageParameters = Map("expression" -> infOrNan, "sourceType" -> "DOUBLE", - "targetType" -> "TIMESTAMP", "ansiConfig" -> SQLConf.ANSI_ENABLED.key) - SparkDateTimeExceptionShims.newSparkDateTimeException(errorClass, messageParameters, - Array.empty, "") - } - - def sqlArrayIndexNotStartAtOneError(): RuntimeException = { - QueryExecutionErrors.invalidIndexOfZeroError(context = null) - } - - override def intervalDivByZeroError(origin: Origin): ArithmeticException = { - QueryExecutionErrors.intervalDividedByZeroError(origin.context) +object RapidsErrorUtils extends RapidsErrorUtilsFor340Plus { + def dynamicPartitionParentError: Throwable = { + new AnalysisException("Dynamic partition cannot be the parent of a static partition.") } -} +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor340Plus.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor340Plus.scala new file mode 100644 index 00000000000..030e3c894e3 --- /dev/null +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor340Plus.scala @@ -0,0 +1,95 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "340"} +{"spark": "341"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.shims + +import org.apache.spark.SparkDateTimeException +import org.apache.spark.sql.catalyst.trees.{Origin, SQLQueryContext} +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} + +trait RapidsErrorUtilsFor340Plus extends RapidsErrorUtilsFor330plus with RapidsQueryErrorUtils { + + def mapKeyNotExistError( + key: String, + keyType: DataType, + origin: Origin): NoSuchElementException = { + throw new UnsupportedOperationException( + "`mapKeyNotExistError` has been removed since Spark 3.4.0. " + ) + } + + def invalidArrayIndexError( + index: Int, + numElements: Int, + isElementAtF: Boolean = false, + context: SQLQueryContext = null): ArrayIndexOutOfBoundsException = { + if (isElementAtF) { + QueryExecutionErrors.invalidElementAtIndexError(index, numElements, context) + } else { + QueryExecutionErrors.invalidArrayIndexError(index, numElements, context) + } + } + + def arithmeticOverflowError( + message: String, + hint: String = "", + errorContext: SQLQueryContext = null): ArithmeticException = { + QueryExecutionErrors.arithmeticOverflowError(message, hint, errorContext) + } + + def cannotChangeDecimalPrecisionError( + value: Decimal, + toType: DecimalType, + context: SQLQueryContext = null): ArithmeticException = { + QueryExecutionErrors.cannotChangeDecimalPrecisionError( + value, toType.precision, toType.scale, context + ) + } + + def overflowInIntegralDivideError(context: SQLQueryContext = null): ArithmeticException = { + QueryExecutionErrors.arithmeticOverflowError( + "Overflow in integral divide", "try_divide", context + ) + } + + def sparkDateTimeException(infOrNan: String): SparkDateTimeException = { + // These are the arguments required by SparkDateTimeException class to create error message. + val errorClass = "CAST_INVALID_INPUT" + val messageParameters = Map("expression" -> infOrNan, "sourceType" -> "DOUBLE", + "targetType" -> "TIMESTAMP", "ansiConfig" -> SQLConf.ANSI_ENABLED.key) + SparkDateTimeExceptionShims.newSparkDateTimeException(errorClass, messageParameters, + Array.empty, "") + } + + def sqlArrayIndexNotStartAtOneError(): RuntimeException = { + QueryExecutionErrors.invalidIndexOfZeroError(context = null) + } + + override def intervalDivByZeroError(origin: Origin): ArithmeticException = { + QueryExecutionErrors.intervalDividedByZeroError(origin.context) + } +} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala new file mode 100644 index 00000000000..730a2f391f1 --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -0,0 +1,31 @@ +/* + * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.shims + +import org.apache.spark.sql.AnalysisException + +object RapidsErrorUtils extends RapidsErrorUtilsFor340Plus { + override def dynamicPartitionParentError: Throwable = { + /** This exception doesn't have a helper method so the errorClass has to be hardcoded */ + throw new AnalysisException( + errorClass = "_LEGACY_ERROR_TEMP_3079", + messageParameters = Map.empty) + } +} \ No newline at end of file From 9cf104567f7aac7130d4b13eb73d108f73f82d08 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 31 May 2024 16:37:19 -0700 Subject: [PATCH 10/15] Fixed imports --- .../spark/sql/rapids/shims/RapidsQueryErrorUtils.scala | 5 ++++- .../apache/spark/sql/rapids/shims/RapidsErrorUtils.scala | 6 +----- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala index 3b2d404c6d6..216d51e6a16 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -44,7 +44,6 @@ package org.apache.spark.sql.rapids.shims import org.apache.hadoop.fs.Path -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types.StructType @@ -119,4 +118,8 @@ trait RapidsQueryErrorUtils { def illegalParquetTypeError(parquetType: String): Throwable = { QueryCompilationErrors.illegalParquetTypeError(parquetType) } + + def dynamicPartitionParentError: Throwable = { + new RapidsAnalysisException("Dynamic partition cannot be the parent of a static partition.") + } } \ No newline at end of file diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index c7d0c3be850..539b5ade8e0 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -26,8 +26,4 @@ package org.apache.spark.sql.rapids.shims import org.apache.spark.AnalysisException, SparkDateTimeException} -object RapidsErrorUtils extends RapidsErrorUtilsFor340Plus { - def dynamicPartitionParentError: Throwable = { - new AnalysisException("Dynamic partition cannot be the parent of a static partition.") - } -} \ No newline at end of file +object RapidsErrorUtils extends RapidsErrorUtilsFor340Plus \ No newline at end of file From 68645edbb91a82d742b571e711a1ad1dbb5233df Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 31 May 2024 16:51:27 -0700 Subject: [PATCH 11/15] Moved the RapidsAnalysisException out of TrampolineUtil --- .../spark/sql/rapids/execution/TrampolineUtil.scala | 11 ++++++----- .../sql/rapids/shims/RapidsQueryErrorUtils.scala | 1 + 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala index d79b5b046f1..588358258d0 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala @@ -242,9 +242,10 @@ object TrampolineUtil { def getSparkHadoopUtilConf: Configuration = SparkHadoopUtil.get.conf - /** - * This class is to only be used when there is no equivalent helper method in - * Spark to throw an AnalysisException. - */ - class RapidsAnalysisException(msg: String) extends AnalysisException(msg) } + +/** + * This class is to only be used when there is no equivalent helper method in + * Spark to throw an AnalysisException. + */ +class RapidsAnalysisException(msg: String) extends AnalysisException(msg) diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala index 216d51e6a16..ab396cef5e7 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -45,6 +45,7 @@ package org.apache.spark.sql.rapids.shims import org.apache.hadoop.fs.Path import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException import org.apache.spark.sql.types.StructType trait RapidsQueryErrorUtils { From 6d8391f95f2a8736b23eaa13bb8a5420d4602b1c Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 31 May 2024 17:16:08 -0700 Subject: [PATCH 12/15] fixed imports --- .../org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 539b5ade8e0..2f875a54314 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -24,6 +24,4 @@ spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims -import org.apache.spark.AnalysisException, SparkDateTimeException} - object RapidsErrorUtils extends RapidsErrorUtilsFor340Plus \ No newline at end of file From 1d70273f7f6fdf0f3f5045f435b6491e466834ce Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Mon, 3 Jun 2024 13:12:44 -0700 Subject: [PATCH 13/15] addressed review comments --- ...GpuInsertIntoHadoopFsRelationCommand.scala | 2 +- .../expressions/GpuRandomExpressions.scala | 2 +- .../sql/rapids/execution/TrampolineUtil.scala | 16 +++- .../rapids/shims/ParquetSchemaClipShims.scala | 6 +- .../rapids/shims/RapidsQueryErrorUtils.scala | 4 +- .../rapids/shims/ParquetSchemaClipShims.scala | 6 +- .../rapids/shims/RapidsQueryErrorUtils.scala | 5 +- .../rapids/shims/ParquetSchemaClipShims.scala | 6 +- .../sql/rapids/GpuFileFormatWriter.scala | 2 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 70 +++++++++++++- .../shims/RapidsErrorUtilsFor340Plus.scala | 95 ------------------- .../sql/rapids/shims/RapidsErrorUtils.scala | 31 ------ 12 files changed, 99 insertions(+), 146 deletions(-) delete mode 100644 sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor340Plus.scala delete mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala index 860a3aa7c2f..ece5ef5acf5 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala @@ -121,7 +121,7 @@ case class GpuInsertIntoHadoopFsRelationCommand( val pathExists = fs.exists(qualifiedOutputPath) (mode, pathExists) match { case (SaveMode.ErrorIfExists, true) => - throw RapidsErrorUtils.dataPathNotExistError(qualifiedOutputPath.toString) + throw RapidsErrorUtils.outputPathAlreadyExistsError(qualifiedOutputPath) case (SaveMode.Overwrite, true) => if (ifPartitionNotExists && matchingPartitions.nonEmpty) { false diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala index 1a2271f4194..3117dd9a8db 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala @@ -52,7 +52,7 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres @transient protected lazy val seed: Long = child match { case GpuLiteral(s, IntegerType) => s.asInstanceOf[Int] case GpuLiteral(s, LongType) => s.asInstanceOf[Long] - case _ => TrampolineUtil.throwAnalysisException( + case _ => TrampolineUtil.throwRapidsAnalysisException( s"Input argument to $prettyName must be an integer, long or null literal.") } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala index 588358258d0..7b7e12e46ab 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala @@ -157,8 +157,14 @@ object TrampolineUtil { TaskContext.get.taskMemoryManager() } - /** Throw a Spark analysis exception */ - def throwAnalysisException(msg: String) = throw new RapidsAnalysisException(msg) + /** + * Throw a Rapids analysis exception + * This method is to only be used to throw errors specific to the + * RAPIDS Accelerator or errors mirroring Spark where a raw + * AnalysisException is thrown directly rather than via an error + * utility class (this should be rare). + */ + def throwRapidsAnalysisException(msg: String) = throw new RapidsAnalysisException(msg) /** Set the task context for the current thread */ def setTaskContext(tc: TaskContext): Unit = TaskContext.setTaskContext(tc) @@ -245,7 +251,9 @@ object TrampolineUtil { } /** - * This class is to only be used when there is no equivalent helper method in - * Spark to throw an AnalysisException. + * This class is to only be used to throw errors specific to the + * RAPIDS Accelerator or errors mirroring Spark where a raw + * AnalysisException is thrown directly rather than via an error + * utility class (this should be rare). */ class RapidsAnalysisException(msg: String) extends AnalysisException(msg) diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index ed20762cc31..8dfcdbd2662 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -65,7 +65,7 @@ object ParquetSchemaClipShims { if (originalType == null) s"$typeName" else s"$typeName ($originalType)" def typeNotSupported() = - TrampolineUtil.throwAnalysisException(s"Parquet type not supported: $typeString") + TrampolineUtil.throwRapidsAnalysisException(s"Parquet type not supported: $typeString") def typeNotImplemented() = throw RapidsErrorUtils.parquetTypeUnsupportedYetError(typeString) @@ -81,7 +81,7 @@ object ParquetSchemaClipShims { val scale = field.getDecimalMetadata.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - TrampolineUtil.throwAnalysisException(s"Invalid decimal precision: $typeName " + + TrampolineUtil.throwRapidsAnalysisException(s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -121,7 +121,7 @@ object ParquetSchemaClipShims { case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - TrampolineUtil.throwAnalysisException( + TrampolineUtil.throwRapidsAnalysisException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala index 79826586197..72c6ff8ad84 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -23,8 +23,10 @@ spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types.StructType trait RapidsQueryErrorUtils { @@ -106,7 +108,7 @@ trait RapidsQueryErrorUtils { } def dynamicPartitionParentError: Throwable = { - new AnalysisException("Dynamic partition cannot be the parent of a static partition.") + TrampolineUtil.throwRapidsAnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) } def tableOrViewAlreadyExistsError(tableName: String): Throwable = { diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index 41ee9527fbe..049d43dbb77 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -83,7 +83,7 @@ object ParquetSchemaClipShims { val scale = decimalLogicalTypeAnnotation.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - TrampolineUtil.throwAnalysisException( + TrampolineUtil.throwRapidsAnalysisException( s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -144,14 +144,14 @@ object ParquetSchemaClipShims { TimestampType case timestamp: TimestampLogicalTypeAnnotation if timestamp.getUnit == TimeUnit.NANOS && ParquetLegacyNanoAsLongShims.legacyParquetNanosAsLong => - TrampolineUtil.throwAnalysisException( + TrampolineUtil.throwRapidsAnalysisException( "GPU does not support spark.sql.legacy.parquet.nanosAsLong") case _ => illegalType() } case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - TrampolineUtil.throwAnalysisException( + TrampolineUtil.throwRapidsAnalysisException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala index ab396cef5e7..7792af7cf22 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -43,9 +43,10 @@ spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.rapids.execution.RapidsAnalysisException +import org.apache.spark.sql.rapids.execution.{RapidsAnalysisException, TrampolineUtil} import org.apache.spark.sql.types.StructType trait RapidsQueryErrorUtils { @@ -121,6 +122,6 @@ trait RapidsQueryErrorUtils { } def dynamicPartitionParentError: Throwable = { - new RapidsAnalysisException("Dynamic partition cannot be the parent of a static partition.") + TrampolineUtil.throwRapidsAnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) } } \ No newline at end of file diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index fe3f8e34fc0..0f5e05a330b 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -126,7 +126,7 @@ object ParquetSchemaClipShims { val scale = decimalLogicalTypeAnnotation.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - TrampolineUtil.throwAnalysisException(s"Invalid decimal precision: $typeName " + + TrampolineUtil.throwRapidsAnalysisException(s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -185,14 +185,14 @@ object ParquetSchemaClipShims { ParquetTimestampAnnotationShims.timestampTypeForMillisOrMicros(timestamp) case timestamp: TimestampLogicalTypeAnnotation if timestamp.getUnit == TimeUnit.NANOS && ParquetLegacyNanoAsLongShims.legacyParquetNanosAsLong => - TrampolineUtil.throwAnalysisException( + TrampolineUtil.throwRapidsAnalysisException( "GPU does not support spark.sql.legacy.parquet.nanosAsLong") case _ => illegalType() } case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - TrampolineUtil.throwAnalysisException( + TrampolineUtil.throwRapidsAnalysisException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index da981c24123..0e7c7b65a41 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -62,7 +62,7 @@ object GpuFileFormatWriter extends Logging { private def verifySchema(format: ColumnarFileFormat, schema: StructType): Unit = { schema.foreach { field => if (!format.supportDataType(field.dataType)) { - TrampolineUtil.throwAnalysisException( + TrampolineUtil.throwRapidsAnalysisException( s"$format data source does not support ${field.dataType.catalogString} data type.") } } diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 2f875a54314..e6f8886f19c 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -21,7 +21,75 @@ {"spark": "343"} {"spark": "350"} {"spark": "351"} +{"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims -object RapidsErrorUtils extends RapidsErrorUtilsFor340Plus \ No newline at end of file +import org.apache.spark.SparkDateTimeException +import org.apache.spark.sql.catalyst.trees.{Origin, SQLQueryContext} +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} + +object RapidsErrorUtils extends RapidsErrorUtilsFor330plus with RapidsQueryErrorUtils { + + def mapKeyNotExistError( + key: String, + keyType: DataType, + origin: Origin): NoSuchElementException = { + throw new UnsupportedOperationException( + "`mapKeyNotExistError` has been removed since Spark 3.4.0. " + ) + } + + def invalidArrayIndexError( + index: Int, + numElements: Int, + isElementAtF: Boolean = false, + context: SQLQueryContext = null): ArrayIndexOutOfBoundsException = { + if (isElementAtF) { + QueryExecutionErrors.invalidElementAtIndexError(index, numElements, context) + } else { + QueryExecutionErrors.invalidArrayIndexError(index, numElements, context) + } + } + + def arithmeticOverflowError( + message: String, + hint: String = "", + errorContext: SQLQueryContext = null): ArithmeticException = { + QueryExecutionErrors.arithmeticOverflowError(message, hint, errorContext) + } + + def cannotChangeDecimalPrecisionError( + value: Decimal, + toType: DecimalType, + context: SQLQueryContext = null): ArithmeticException = { + QueryExecutionErrors.cannotChangeDecimalPrecisionError( + value, toType.precision, toType.scale, context + ) + } + + def overflowInIntegralDivideError(context: SQLQueryContext = null): ArithmeticException = { + QueryExecutionErrors.arithmeticOverflowError( + "Overflow in integral divide", "try_divide", context + ) + } + + def sparkDateTimeException(infOrNan: String): SparkDateTimeException = { + // These are the arguments required by SparkDateTimeException class to create error message. + val errorClass = "CAST_INVALID_INPUT" + val messageParameters = Map("expression" -> infOrNan, "sourceType" -> "DOUBLE", + "targetType" -> "TIMESTAMP", "ansiConfig" -> SQLConf.ANSI_ENABLED.key) + SparkDateTimeExceptionShims.newSparkDateTimeException(errorClass, messageParameters, + Array.empty, "") + } + + def sqlArrayIndexNotStartAtOneError(): RuntimeException = { + QueryExecutionErrors.invalidIndexOfZeroError(context = null) + } + + override def intervalDivByZeroError(origin: Origin): ArithmeticException = { + QueryExecutionErrors.intervalDividedByZeroError(origin.context) + } +} diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor340Plus.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor340Plus.scala deleted file mode 100644 index 030e3c894e3..00000000000 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor340Plus.scala +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Copyright (c) 2024, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/*** spark-rapids-shim-json-lines -{"spark": "340"} -{"spark": "341"} -{"spark": "342"} -{"spark": "343"} -{"spark": "350"} -{"spark": "351"} -{"spark": "400"} -spark-rapids-shim-json-lines ***/ -package org.apache.spark.sql.rapids.shims - -import org.apache.spark.SparkDateTimeException -import org.apache.spark.sql.catalyst.trees.{Origin, SQLQueryContext} -import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} - -trait RapidsErrorUtilsFor340Plus extends RapidsErrorUtilsFor330plus with RapidsQueryErrorUtils { - - def mapKeyNotExistError( - key: String, - keyType: DataType, - origin: Origin): NoSuchElementException = { - throw new UnsupportedOperationException( - "`mapKeyNotExistError` has been removed since Spark 3.4.0. " - ) - } - - def invalidArrayIndexError( - index: Int, - numElements: Int, - isElementAtF: Boolean = false, - context: SQLQueryContext = null): ArrayIndexOutOfBoundsException = { - if (isElementAtF) { - QueryExecutionErrors.invalidElementAtIndexError(index, numElements, context) - } else { - QueryExecutionErrors.invalidArrayIndexError(index, numElements, context) - } - } - - def arithmeticOverflowError( - message: String, - hint: String = "", - errorContext: SQLQueryContext = null): ArithmeticException = { - QueryExecutionErrors.arithmeticOverflowError(message, hint, errorContext) - } - - def cannotChangeDecimalPrecisionError( - value: Decimal, - toType: DecimalType, - context: SQLQueryContext = null): ArithmeticException = { - QueryExecutionErrors.cannotChangeDecimalPrecisionError( - value, toType.precision, toType.scale, context - ) - } - - def overflowInIntegralDivideError(context: SQLQueryContext = null): ArithmeticException = { - QueryExecutionErrors.arithmeticOverflowError( - "Overflow in integral divide", "try_divide", context - ) - } - - def sparkDateTimeException(infOrNan: String): SparkDateTimeException = { - // These are the arguments required by SparkDateTimeException class to create error message. - val errorClass = "CAST_INVALID_INPUT" - val messageParameters = Map("expression" -> infOrNan, "sourceType" -> "DOUBLE", - "targetType" -> "TIMESTAMP", "ansiConfig" -> SQLConf.ANSI_ENABLED.key) - SparkDateTimeExceptionShims.newSparkDateTimeException(errorClass, messageParameters, - Array.empty, "") - } - - def sqlArrayIndexNotStartAtOneError(): RuntimeException = { - QueryExecutionErrors.invalidIndexOfZeroError(context = null) - } - - override def intervalDivByZeroError(origin: Origin): ArithmeticException = { - QueryExecutionErrors.intervalDividedByZeroError(origin.context) - } -} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala deleted file mode 100644 index 730a2f391f1..00000000000 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/*** spark-rapids-shim-json-lines -{"spark": "400"} -spark-rapids-shim-json-lines ***/ -package org.apache.spark.sql.rapids.shims - -import org.apache.spark.sql.AnalysisException - -object RapidsErrorUtils extends RapidsErrorUtilsFor340Plus { - override def dynamicPartitionParentError: Throwable = { - /** This exception doesn't have a helper method so the errorClass has to be hardcoded */ - throw new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_3079", - messageParameters = Map.empty) - } -} \ No newline at end of file From 45718e9da5dfe44bc23450cc6a25e39e5bea5d60 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Mon, 3 Jun 2024 13:46:51 -0700 Subject: [PATCH 14/15] fixed unused import --- .../apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala index 7792af7cf22..cf8b0bcba7b 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -46,7 +46,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.rapids.execution.{RapidsAnalysisException, TrampolineUtil} +import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types.StructType trait RapidsQueryErrorUtils { From 65cb71a6d251b8c8a9991d729d958a2aade75e93 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Mon, 3 Jun 2024 15:36:20 -0700 Subject: [PATCH 15/15] Removed the TrampolineUtil method for throwing RapidsAnalysisException --- .../catalyst/expressions/GpuRandomExpressions.scala | 4 ++-- .../spark/sql/rapids/execution/TrampolineUtil.scala | 9 --------- .../spark/rapids/shims/ParquetSchemaClipShims.scala | 8 ++++---- .../spark/sql/rapids/shims/RapidsQueryErrorUtils.scala | 4 ++-- .../spark/rapids/shims/ParquetSchemaClipShims.scala | 8 ++++---- .../spark/sql/rapids/shims/RapidsQueryErrorUtils.scala | 4 ++-- .../spark/rapids/shims/ParquetSchemaClipShims.scala | 8 ++++---- .../apache/spark/sql/rapids/GpuFileFormatWriter.scala | 4 ++-- 8 files changed, 20 insertions(+), 29 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala index 3117dd9a8db..f9d0be81505 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala @@ -24,7 +24,7 @@ import com.nvidia.spark.rapids.shims.ShimUnaryExpression import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionWithRandomSeed} -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils @@ -52,7 +52,7 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres @transient protected lazy val seed: Long = child match { case GpuLiteral(s, IntegerType) => s.asInstanceOf[Int] case GpuLiteral(s, LongType) => s.asInstanceOf[Long] - case _ => TrampolineUtil.throwRapidsAnalysisException( + case _ => throw new RapidsAnalysisException( s"Input argument to $prettyName must be an integer, long or null literal.") } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala index 7b7e12e46ab..8a88cc4024d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala @@ -157,15 +157,6 @@ object TrampolineUtil { TaskContext.get.taskMemoryManager() } - /** - * Throw a Rapids analysis exception - * This method is to only be used to throw errors specific to the - * RAPIDS Accelerator or errors mirroring Spark where a raw - * AnalysisException is thrown directly rather than via an error - * utility class (this should be rare). - */ - def throwRapidsAnalysisException(msg: String) = throw new RapidsAnalysisException(msg) - /** Set the task context for the current thread */ def setTaskContext(tc: TaskContext): Unit = TaskContext.setTaskContext(tc) diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index 8dfcdbd2662..4d6d4967a80 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -26,7 +26,7 @@ import org.apache.parquet.schema.OriginalType._ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ @@ -65,7 +65,7 @@ object ParquetSchemaClipShims { if (originalType == null) s"$typeName" else s"$typeName ($originalType)" def typeNotSupported() = - TrampolineUtil.throwRapidsAnalysisException(s"Parquet type not supported: $typeString") + throw new RapidsAnalysisException(s"Parquet type not supported: $typeString") def typeNotImplemented() = throw RapidsErrorUtils.parquetTypeUnsupportedYetError(typeString) @@ -81,7 +81,7 @@ object ParquetSchemaClipShims { val scale = field.getDecimalMetadata.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - TrampolineUtil.throwRapidsAnalysisException(s"Invalid decimal precision: $typeName " + + throw new RapidsAnalysisException(s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -121,7 +121,7 @@ object ParquetSchemaClipShims { case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - TrampolineUtil.throwRapidsAnalysisException( + throw new RapidsAnalysisException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala index 72c6ff8ad84..266cb4ef54f 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException import org.apache.spark.sql.types.StructType trait RapidsQueryErrorUtils { @@ -108,7 +108,7 @@ trait RapidsQueryErrorUtils { } def dynamicPartitionParentError: Throwable = { - TrampolineUtil.throwRapidsAnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) + throw new RapidsAnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) } def tableOrViewAlreadyExistsError(tableName: String): Throwable = { diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index 049d43dbb77..bba205f267f 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -29,7 +29,7 @@ import org.apache.parquet.schema.LogicalTypeAnnotation._ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ @@ -83,7 +83,7 @@ object ParquetSchemaClipShims { val scale = decimalLogicalTypeAnnotation.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - TrampolineUtil.throwRapidsAnalysisException( + throw new RapidsAnalysisException( s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -144,14 +144,14 @@ object ParquetSchemaClipShims { TimestampType case timestamp: TimestampLogicalTypeAnnotation if timestamp.getUnit == TimeUnit.NANOS && ParquetLegacyNanoAsLongShims.legacyParquetNanosAsLong => - TrampolineUtil.throwRapidsAnalysisException( + throw new RapidsAnalysisException( "GPU does not support spark.sql.legacy.parquet.nanosAsLong") case _ => illegalType() } case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - TrampolineUtil.throwRapidsAnalysisException( + throw new RapidsAnalysisException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala index cf8b0bcba7b..dbc4145ee54 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -46,7 +46,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException import org.apache.spark.sql.types.StructType trait RapidsQueryErrorUtils { @@ -122,6 +122,6 @@ trait RapidsQueryErrorUtils { } def dynamicPartitionParentError: Throwable = { - TrampolineUtil.throwRapidsAnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) + throw new RapidsAnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) } } \ No newline at end of file diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index 0f5e05a330b..8c395274e07 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -44,7 +44,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport.containsFieldIds import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ @@ -126,7 +126,7 @@ object ParquetSchemaClipShims { val scale = decimalLogicalTypeAnnotation.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - TrampolineUtil.throwRapidsAnalysisException(s"Invalid decimal precision: $typeName " + + throw new RapidsAnalysisException(s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -185,14 +185,14 @@ object ParquetSchemaClipShims { ParquetTimestampAnnotationShims.timestampTypeForMillisOrMicros(timestamp) case timestamp: TimestampLogicalTypeAnnotation if timestamp.getUnit == TimeUnit.NANOS && ParquetLegacyNanoAsLongShims.legacyParquetNanosAsLong => - TrampolineUtil.throwRapidsAnalysisException( + throw new RapidsAnalysisException( "GPU does not support spark.sql.legacy.parquet.nanosAsLong") case _ => illegalType() } case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - TrampolineUtil.throwRapidsAnalysisException( + throw new RapidsAnalysisException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index 0e7c7b65a41..e7b3561f5fd 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -51,7 +51,7 @@ import org.apache.spark.sql.connector.write.WriterCommitMessage import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.datasources.{GpuWriteFiles, GpuWriteFilesExec, GpuWriteFilesSpec, WriteTaskResult, WriteTaskStats} import org.apache.spark.sql.execution.datasources.FileFormatWriter.OutputSpec -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -62,7 +62,7 @@ object GpuFileFormatWriter extends Logging { private def verifySchema(format: ColumnarFileFormat, schema: StructType): Unit = { schema.foreach { field => if (!format.supportDataType(field.dataType)) { - TrampolineUtil.throwRapidsAnalysisException( + throw new RapidsAnalysisException( s"$format data source does not support ${field.dataType.catalogString} data type.") } }